From 2d2750454c1567ba0e7a3af1401b9a3b4cbfda1f Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Thu, 20 Jul 2017 11:18:05 +0900 Subject: [PATCH 1/5] Support spark-2.2 --- pom.xml | 11 + spark/spark-2.2/bin/mvn-zinc | 99 ++ spark/spark-2.2/extra-src/README.md | 1 + .../org/apache/spark/sql/hive/HiveShim.scala | 279 +++ spark/spark-2.2/pom.xml | 269 +++ .../hivemall/xgboost/XGBoostOptions.scala | 59 + ...pache.spark.sql.sources.DataSourceRegister | 1 + .../src/main/resources/log4j.properties | 12 + .../hivemall/tools/RegressionDatagen.scala | 67 + .../sql/catalyst/expressions/EachTopK.scala | 133 ++ .../sql/catalyst/plans/logical/JoinTopK.scala | 68 + .../utils/InternalRowPriorityQueue.scala | 76 + .../sql/execution/UserProvidedPlanner.scala | 83 + .../datasources/csv/csvExpressions.scala | 169 ++ .../joins/ShuffledHashJoinTopKExec.scala | 405 +++++ .../sql/hive/HivemallGroupedDataset.scala | 304 ++++ .../apache/spark/sql/hive/HivemallOps.scala | 1538 +++++++++++++++++ .../apache/spark/sql/hive/HivemallUtils.scala | 146 ++ .../sql/hive/internal/HivemallOpsImpl.scala | 79 + .../sql/hive/source/XGBoostFileFormat.scala | 163 ++ .../src/test/resources/data/files/README.md | 3 + .../src/test/resources/data/files/complex.seq | 0 .../test/resources/data/files/episodes.avro | 0 .../src/test/resources/data/files/json.txt | 0 .../src/test/resources/data/files/kv1.txt | 0 .../src/test/resources/data/files/kv3.txt | 0 .../src/test/resources/log4j.properties | 7 + .../hivemall/mix/server/MixServerSuite.scala | 124 ++ .../tools/RegressionDatagenSuite.scala | 33 + .../org/apache/spark/SparkFunSuite.scala | 51 + .../feature/HivemallLabeledPointSuite.scala | 36 + .../org/apache/spark/sql/QueryTest.scala | 360 ++++ .../spark/sql/catalyst/plans/PlanTest.scala | 135 ++ .../execution/benchmark/BenchmarkBase.scala | 56 + .../apache/spark/sql/hive/HiveUdfSuite.scala | 161 ++ .../spark/sql/hive/HivemallOpsSuite.scala | 961 ++++++++++ .../spark/sql/hive/ModelMixingSuite.scala | 286 +++ .../apache/spark/sql/hive/XGBoostSuite.scala | 151 ++ .../sql/hive/benchmark/MiscBenchmark.scala | 268 +++ .../hive/test/HivemallFeatureQueryTest.scala | 113 ++ .../sql/hive/test/TestHiveSingleton.scala | 39 + .../apache/spark/sql/test/SQLTestData.scala | 315 ++++ .../apache/spark/sql/test/SQLTestUtils.scala | 336 ++++ .../spark/sql/test/VectorQueryTest.scala | 89 + .../HivemallOpsWithFeatureSuite.scala | 155 ++ .../org/apache/spark/test/TestUtils.scala | 65 + 46 files changed, 7706 insertions(+) create mode 100755 spark/spark-2.2/bin/mvn-zinc create mode 100644 spark/spark-2.2/extra-src/README.md create mode 100644 spark/spark-2.2/extra-src/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala create mode 100644 spark/spark-2.2/pom.xml create mode 100644 spark/spark-2.2/src/main/java/hivemall/xgboost/XGBoostOptions.scala create mode 100644 spark/spark-2.2/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister create mode 100644 spark/spark-2.2/src/main/resources/log4j.properties create mode 100644 spark/spark-2.2/src/main/scala/hivemall/tools/RegressionDatagen.scala create mode 100644 spark/spark-2.2/src/main/scala/org/apache/spark/sql/catalyst/expressions/EachTopK.scala create mode 100644 spark/spark-2.2/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/JoinTopK.scala create mode 100644 spark/spark-2.2/src/main/scala/org/apache/spark/sql/catalyst/utils/InternalRowPriorityQueue.scala create mode 100644 spark/spark-2.2/src/main/scala/org/apache/spark/sql/execution/UserProvidedPlanner.scala create mode 100644 spark/spark-2.2/src/main/scala/org/apache/spark/sql/execution/datasources/csv/csvExpressions.scala create mode 100644 spark/spark-2.2/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinTopKExec.scala create mode 100644 spark/spark-2.2/src/main/scala/org/apache/spark/sql/hive/HivemallGroupedDataset.scala create mode 100644 spark/spark-2.2/src/main/scala/org/apache/spark/sql/hive/HivemallOps.scala create mode 100644 spark/spark-2.2/src/main/scala/org/apache/spark/sql/hive/HivemallUtils.scala create mode 100644 spark/spark-2.2/src/main/scala/org/apache/spark/sql/hive/internal/HivemallOpsImpl.scala create mode 100644 spark/spark-2.2/src/main/scala/org/apache/spark/sql/hive/source/XGBoostFileFormat.scala create mode 100644 spark/spark-2.2/src/test/resources/data/files/README.md create mode 100644 spark/spark-2.2/src/test/resources/data/files/complex.seq create mode 100644 spark/spark-2.2/src/test/resources/data/files/episodes.avro create mode 100644 spark/spark-2.2/src/test/resources/data/files/json.txt create mode 100644 spark/spark-2.2/src/test/resources/data/files/kv1.txt create mode 100644 spark/spark-2.2/src/test/resources/data/files/kv3.txt create mode 100644 spark/spark-2.2/src/test/resources/log4j.properties create mode 100644 spark/spark-2.2/src/test/scala/hivemall/mix/server/MixServerSuite.scala create mode 100644 spark/spark-2.2/src/test/scala/hivemall/tools/RegressionDatagenSuite.scala create mode 100644 spark/spark-2.2/src/test/scala/org/apache/spark/SparkFunSuite.scala create mode 100644 spark/spark-2.2/src/test/scala/org/apache/spark/ml/feature/HivemallLabeledPointSuite.scala create mode 100644 spark/spark-2.2/src/test/scala/org/apache/spark/sql/QueryTest.scala create mode 100644 spark/spark-2.2/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala create mode 100644 spark/spark-2.2/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkBase.scala create mode 100644 spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/HiveUdfSuite.scala create mode 100644 spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/HivemallOpsSuite.scala create mode 100644 spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/ModelMixingSuite.scala create mode 100644 spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/XGBoostSuite.scala create mode 100644 spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/benchmark/MiscBenchmark.scala create mode 100644 spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/test/HivemallFeatureQueryTest.scala create mode 100644 spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala create mode 100644 spark/spark-2.2/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala create mode 100644 spark/spark-2.2/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala create mode 100644 spark/spark-2.2/src/test/scala/org/apache/spark/sql/test/VectorQueryTest.scala create mode 100644 spark/spark-2.2/src/test/scala/org/apache/spark/streaming/HivemallOpsWithFeatureSuite.scala create mode 100644 spark/spark-2.2/src/test/scala/org/apache/spark/test/TestUtils.scala diff --git a/pom.xml b/pom.xml index 3d7040cb9..453209176 100644 --- a/pom.xml +++ b/pom.xml @@ -266,6 +266,17 @@ + + spark-2.2 + + spark/spark-2.2 + spark/spark-common + + + 2.2.0 + 2.2 + + spark-2.1 diff --git a/spark/spark-2.2/bin/mvn-zinc b/spark/spark-2.2/bin/mvn-zinc new file mode 100755 index 000000000..759b0a56d --- /dev/null +++ b/spark/spark-2.2/bin/mvn-zinc @@ -0,0 +1,99 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# Copyed from commit 48682f6bf663e54cb63b7e95a4520d34b6fa890b in Apache Spark + +# Determine the current working directory +_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" +# Preserve the calling directory +_CALLING_DIR="$(pwd)" +# Options used during compilation +_COMPILE_JVM_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m" + +# Installs any application tarball given a URL, the expected tarball name, +# and, optionally, a checkable binary path to determine if the binary has +# already been installed +## Arg1 - URL +## Arg2 - Tarball Name +## Arg3 - Checkable Binary +install_app() { + local remote_tarball="$1/$2" + local local_tarball="${_DIR}/$2" + local binary="${_DIR}/$3" + local curl_opts="--progress-bar -L" + local wget_opts="--progress=bar:force ${wget_opts}" + + if [ -z "$3" -o ! -f "$binary" ]; then + # check if we already have the tarball + # check if we have curl installed + # download application + [ ! -f "${local_tarball}" ] && [ $(command -v curl) ] && \ + echo "exec: curl ${curl_opts} ${remote_tarball}" 1>&2 && \ + curl ${curl_opts} "${remote_tarball}" > "${local_tarball}" + # if the file still doesn't exist, lets try `wget` and cross our fingers + [ ! -f "${local_tarball}" ] && [ $(command -v wget) ] && \ + echo "exec: wget ${wget_opts} ${remote_tarball}" 1>&2 && \ + wget ${wget_opts} -O "${local_tarball}" "${remote_tarball}" + # if both were unsuccessful, exit + [ ! -f "${local_tarball}" ] && \ + echo -n "ERROR: Cannot download $2 with cURL or wget; " && \ + echo "please install manually and try again." && \ + exit 2 + cd "${_DIR}" && tar -xzf "$2" + rm -rf "$local_tarball" + fi +} + +# Install zinc under the bin/ folder +install_zinc() { + local zinc_path="zinc-0.3.9/bin/zinc" + [ ! -f "${_DIR}/${zinc_path}" ] && ZINC_INSTALL_FLAG=1 + install_app \ + "http://downloads.typesafe.com/zinc/0.3.9" \ + "zinc-0.3.9.tgz" \ + "${zinc_path}" + ZINC_BIN="${_DIR}/${zinc_path}" +} + +# Setup healthy defaults for the Zinc port if none were provided from +# the environment +ZINC_PORT=${ZINC_PORT:-"3030"} + +# Install Zinc for the bin/ +install_zinc + +# Reset the current working directory +cd "${_CALLING_DIR}" + +# Now that zinc is ensured to be installed, check its status and, if its +# not running or just installed, start it +if [ ! -f "${ZINC_BIN}" ]; then + exit -1 +fi +if [ -n "${ZINC_INSTALL_FLAG}" -o -z "`"${ZINC_BIN}" -status -port ${ZINC_PORT}`" ]; then + export ZINC_OPTS=${ZINC_OPTS:-"$_COMPILE_JVM_OPTS"} + "${ZINC_BIN}" -shutdown -port ${ZINC_PORT} + "${ZINC_BIN}" -start -port ${ZINC_PORT} &>/dev/null +fi + +# Set any `mvn` options if not already present +export MAVEN_OPTS=${MAVEN_OPTS:-"$_COMPILE_JVM_OPTS"} + +# Last, call the `mvn` command as usual +mvn -DzincPort=${ZINC_PORT} "$@" diff --git a/spark/spark-2.2/extra-src/README.md b/spark/spark-2.2/extra-src/README.md new file mode 100644 index 000000000..1d89d0a84 --- /dev/null +++ b/spark/spark-2.2/extra-src/README.md @@ -0,0 +1 @@ +Copyed from the spark v2.2.0 release. diff --git a/spark/spark-2.2/extra-src/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala b/spark/spark-2.2/extra-src/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala new file mode 100644 index 000000000..9e9894803 --- /dev/null +++ b/spark/spark-2.2/extra-src/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala @@ -0,0 +1,279 @@ +/* + * 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.{InputStream, OutputStream} +import java.rmi.server.UID + +import scala.collection.JavaConverters._ +import scala.language.implicitConversions +import scala.reflect.ClassTag + +import com.google.common.base.Objects +import org.apache.avro.Schema +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.ql.exec.{UDF, Utilities} +import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFMacro +import org.apache.hadoop.hive.serde2.ColumnProjectionUtils +import org.apache.hadoop.hive.serde2.avro.{AvroGenericRecordWritable, AvroSerdeUtils} +import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector +import org.apache.hadoop.io.Writable +import org.apache.hive.com.esotericsoftware.kryo.Kryo +import org.apache.hive.com.esotericsoftware.kryo.io.{Input, Output} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.types.Decimal +import org.apache.spark.util.Utils + +private[hive] object HiveShim { + // Precision and scale to pass for unlimited decimals; these are the same as the precision and + // scale Hive 0.13 infers for BigDecimals from sources that don't specify them (e.g. UDFs) + val UNLIMITED_DECIMAL_PRECISION = 38 + val UNLIMITED_DECIMAL_SCALE = 18 + val HIVE_GENERIC_UDF_MACRO_CLS = "org.apache.hadoop.hive.ql.udf.generic.GenericUDFMacro" + + /* + * This function in hive-0.13 become private, but we have to do this to walkaround hive bug + */ + private def appendReadColumnNames(conf: Configuration, cols: Seq[String]) { + val old: String = conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, "") + val result: StringBuilder = new StringBuilder(old) + var first: Boolean = old.isEmpty + + for (col <- cols) { + if (first) { + first = false + } else { + result.append(',') + } + result.append(col) + } + conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, result.toString) + } + + /* + * Cannot use ColumnProjectionUtils.appendReadColumns directly, if ids is null + */ + def appendReadColumns(conf: Configuration, ids: Seq[Integer], names: Seq[String]) { + if (ids != null) { + ColumnProjectionUtils.appendReadColumns(conf, ids.asJava) + } + if (names != null) { + appendReadColumnNames(conf, names) + } + } + + /* + * Bug introduced in hive-0.13. AvroGenericRecordWritable has a member recordReaderID that + * is needed to initialize before serialization. + */ + def prepareWritable(w: Writable, serDeProps: Seq[(String, String)]): Writable = { + w match { + case w: AvroGenericRecordWritable => + w.setRecordReaderID(new UID()) + // In Hive 1.1, the record's schema may need to be initialized manually or a NPE will + // be thrown. + if (w.getFileSchema() == null) { + serDeProps + .find(_._1 == AvroSerdeUtils.AvroTableProperties.SCHEMA_LITERAL.getPropName()) + .foreach { kv => + w.setFileSchema(new Schema.Parser().parse(kv._2)) + } + } + case _ => + } + w + } + + def toCatalystDecimal(hdoi: HiveDecimalObjectInspector, data: Any): Decimal = { + if (hdoi.preferWritable()) { + Decimal(hdoi.getPrimitiveWritableObject(data).getHiveDecimal().bigDecimalValue, + hdoi.precision(), hdoi.scale()) + } else { + Decimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue(), hdoi.precision(), hdoi.scale()) + } + } + + /** + * This class provides the UDF creation and also the UDF instance serialization and + * de-serialization cross process boundary. + * + * Detail discussion can be found at https://github.com/apache/spark/pull/3640 + * + * @param functionClassName UDF class name + * @param instance optional UDF instance which contains additional information (for macro) + */ + private[hive] case class HiveFunctionWrapper(var functionClassName: String, + private var instance: AnyRef = null) extends java.io.Externalizable { + + // for Serialization + def this() = this(null) + + override def hashCode(): Int = { + if (functionClassName == HIVE_GENERIC_UDF_MACRO_CLS) { + Objects.hashCode(functionClassName, instance.asInstanceOf[GenericUDFMacro].getBody()) + } else { + functionClassName.hashCode() + } + } + + override def equals(other: Any): Boolean = other match { + case a: HiveFunctionWrapper if functionClassName == a.functionClassName => + // In case of udf macro, check to make sure they point to the same underlying UDF + if (functionClassName == HIVE_GENERIC_UDF_MACRO_CLS) { + a.instance.asInstanceOf[GenericUDFMacro].getBody() == + instance.asInstanceOf[GenericUDFMacro].getBody() + } else { + true + } + case _ => false + } + + @transient + def deserializeObjectByKryo[T: ClassTag]( + kryo: Kryo, + in: InputStream, + clazz: Class[_]): T = { + val inp = new Input(in) + val t: T = kryo.readObject(inp, clazz).asInstanceOf[T] + inp.close() + t + } + + @transient + def serializeObjectByKryo( + kryo: Kryo, + plan: Object, + out: OutputStream) { + val output: Output = new Output(out) + kryo.writeObject(output, plan) + output.close() + } + + def deserializePlan[UDFType](is: java.io.InputStream, clazz: Class[_]): UDFType = { + deserializeObjectByKryo(Utilities.runtimeSerializationKryo.get(), is, clazz) + .asInstanceOf[UDFType] + } + + def serializePlan(function: AnyRef, out: java.io.OutputStream): Unit = { + serializeObjectByKryo(Utilities.runtimeSerializationKryo.get(), function, out) + } + + def writeExternal(out: java.io.ObjectOutput) { + // output the function name + out.writeUTF(functionClassName) + + // Write a flag if instance is null or not + out.writeBoolean(instance != null) + if (instance != null) { + // Some of the UDF are serializable, but some others are not + // Hive Utilities can handle both cases + val baos = new java.io.ByteArrayOutputStream() + serializePlan(instance, baos) + val functionInBytes = baos.toByteArray + + // output the function bytes + out.writeInt(functionInBytes.length) + out.write(functionInBytes, 0, functionInBytes.length) + } + } + + def readExternal(in: java.io.ObjectInput) { + // read the function name + functionClassName = in.readUTF() + + if (in.readBoolean()) { + // if the instance is not null + // read the function in bytes + val functionInBytesLength = in.readInt() + val functionInBytes = new Array[Byte](functionInBytesLength) + in.readFully(functionInBytes) + + // deserialize the function object via Hive Utilities + instance = deserializePlan[AnyRef](new java.io.ByteArrayInputStream(functionInBytes), + Utils.getContextOrSparkClassLoader.loadClass(functionClassName)) + } + } + + def createFunction[UDFType <: AnyRef](): UDFType = { + if (instance != null) { + instance.asInstanceOf[UDFType] + } else { + val func = Utils.getContextOrSparkClassLoader + .loadClass(functionClassName).newInstance.asInstanceOf[UDFType] + if (!func.isInstanceOf[UDF]) { + // We cache the function if it's no the Simple UDF, + // as we always have to create new instance for Simple UDF + instance = func + } + func + } + } + } + + /* + * Bug introduced in hive-0.13. FileSinkDesc is serializable, but its member path is not. + * Fix it through wrapper. + */ + implicit def wrapperToFileSinkDesc(w: ShimFileSinkDesc): FileSinkDesc = { + val f = new FileSinkDesc(new Path(w.dir), w.tableInfo, w.compressed) + f.setCompressCodec(w.compressCodec) + f.setCompressType(w.compressType) + f.setTableInfo(w.tableInfo) + f.setDestTableId(w.destTableId) + f + } + + /* + * Bug introduced in hive-0.13. FileSinkDesc is serializable, but its member path is not. + * Fix it through wrapper. + */ + private[hive] class ShimFileSinkDesc( + var dir: String, + var tableInfo: TableDesc, + var compressed: Boolean) + extends Serializable with Logging { + var compressCodec: String = _ + var compressType: String = _ + var destTableId: Int = _ + + def setCompressed(compressed: Boolean) { + this.compressed = compressed + } + + def getDirName(): String = dir + + def setDestTableId(destTableId: Int) { + this.destTableId = destTableId + } + + def setTableInfo(tableInfo: TableDesc) { + this.tableInfo = tableInfo + } + + def setCompressCodec(intermediateCompressorCodec: String) { + compressCodec = intermediateCompressorCodec + } + + def setCompressType(intermediateCompressType: String) { + compressType = intermediateCompressType + } + } +} diff --git a/spark/spark-2.2/pom.xml b/spark/spark-2.2/pom.xml new file mode 100644 index 000000000..85a296f77 --- /dev/null +++ b/spark/spark-2.2/pom.xml @@ -0,0 +1,269 @@ + + + 4.0.0 + + + io.github.myui + hivemall + 0.4.2-rc.2 + ../../pom.xml + + + hivemall-spark + Hivemall on Spark 2.2 + jar + + + 64m + 512m + 512m + ${project.parent.basedir} + + + + + + io.github.myui + hivemall-core + ${project.version} + compile + + + io.github.myui + hivemall-xgboost + ${project.version} + compile + + + io.github.myui + hivemall-spark-common + ${project.version} + compile + + + + + org.scala-lang + scala-library + ${scala.version} + compile + + + org.apache.commons + commons-compress + 1.8 + compile + + + + + org.apache.spark + spark-core_${scala.binary.version} + ${spark.version} + provided + + + org.apache.spark + spark-sql_${scala.binary.version} + ${spark.version} + provided + + + org.apache.spark + spark-hive_${scala.binary.version} + ${spark.version} + provided + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${spark.version} + provided + + + org.apache.spark + spark-mllib_${scala.binary.version} + ${spark.version} + provided + + + + + io.github.myui + hivemall-mixserv + ${project.version} + test + + + org.xerial + xerial-core + 3.2.3 + test + + + org.scalatest + scalatest_${scala.binary.version} + 2.2.4 + test + + + + + target + target/classes + ${project.artifactId}-${spark.binary.version}_${scala.binary.version}-${project.version} + target/test-classes + + + + net.alchim31.maven + scala-maven-plugin + 3.2.2 + + + scala-compile-first + process-resources + + compile + + + + scala-test-compile-first + process-test-resources + + testCompile + + + + + ${scala.version} + incremental + true + + -unchecked + -deprecation + + + + + -Xms1024m + -Xmx1024m + -XX:PermSize=${PermGen} + -XX:MaxPermSize=${MaxPermGen} + -XX:ReservedCodeCacheSize=${CodeCacheSize} + + + + + + org.apache.maven.plugins + maven-jar-plugin + 2.5 + + ${project.artifactId}-${spark.binary.version}_${scala.binary.version}-${project.version} + ${project.parent.build.directory} + + + + + org.apache.maven.plugins + maven-shade-plugin + 2.3 + + + jar-with-dependencies + package + + shade + + + ${project.artifactId}-${spark.binary.version}_${scala.binary.version}-${project.version}-with-dependencies + ${project.parent.build.directory} + false + false + + + io.github.myui:hivemall-core + io.github.myui:hivemall-xgboost + io.github.myui:hivemall-spark-common + com.github.haifengl:smile-core + com.github.haifengl:smile-math + com.github.haifengl:smile-data + ml.dmlc:xgboost4j + com.esotericsoftware.kryo:kryo + + + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.7 + + true + + + + + org.scalatest + scalatest-maven-plugin + 1.0 + + ${project.build.directory}/surefire-reports + . + SparkTestSuite.txt + -ea -Xmx2g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=${CodeCacheSize} + + + 1 + ${scala.binary.version} + 1 + ${env.JAVA_HOME} + + + file:src/test/resources/log4j.properties + test + true + ${project.build.directory}/tmp + 1 + false + false + true + + __not_used__ + + ${test.exclude.tags} + + + + test + + test + + + + + + + diff --git a/spark/spark-2.2/src/main/java/hivemall/xgboost/XGBoostOptions.scala b/spark/spark-2.2/src/main/java/hivemall/xgboost/XGBoostOptions.scala new file mode 100644 index 000000000..3e0f274aa --- /dev/null +++ b/spark/spark-2.2/src/main/java/hivemall/xgboost/XGBoostOptions.scala @@ -0,0 +1,59 @@ +/* + * 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 hivemall.xgboost + +import scala.collection.mutable + +import org.apache.commons.cli.Options +import org.apache.spark.annotation.AlphaComponent + +/** + * :: AlphaComponent :: + * An utility class to generate a sequence of options used in XGBoost. + */ +@AlphaComponent +case class XGBoostOptions() { + private val params: mutable.Map[String, String] = mutable.Map.empty + private val options: Options = { + new XGBoostUDTF() { + def options(): Options = super.getOptions() + }.options() + } + + private def isValidKey(key: String): Boolean = { + // TODO: Is there another way to handle all the XGBoost options? + options.hasOption(key) || key == "num_class" + } + + def set(key: String, value: String): XGBoostOptions = { + require(isValidKey(key), s"non-existing key detected in XGBoost options: ${key}") + params.put(key, value) + this + } + + def help(): Unit = { + import scala.collection.JavaConversions._ + options.getOptions.map { case option => println(option) } + } + + override def toString(): String = { + params.map { case (key, value) => s"-$key $value" }.mkString(" ") + } +} diff --git a/spark/spark-2.2/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/spark/spark-2.2/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister new file mode 100644 index 000000000..b49e20a2a --- /dev/null +++ b/spark/spark-2.2/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -0,0 +1 @@ +org.apache.spark.sql.hive.source.XGBoostFileFormat diff --git a/spark/spark-2.2/src/main/resources/log4j.properties b/spark/spark-2.2/src/main/resources/log4j.properties new file mode 100644 index 000000000..72bf5b62d --- /dev/null +++ b/spark/spark-2.2/src/main/resources/log4j.properties @@ -0,0 +1,12 @@ +# Set everything to be logged to the console +log4j.rootCategory=INFO, console +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n + +# Settings to quiet third party logs that are too verbose +log4j.logger.org.eclipse.jetty=INFO +log4j.logger.org.eclipse.jetty.util.component.AbstractLifeCycle=ERROR +log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO +log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO diff --git a/spark/spark-2.2/src/main/scala/hivemall/tools/RegressionDatagen.scala b/spark/spark-2.2/src/main/scala/hivemall/tools/RegressionDatagen.scala new file mode 100644 index 000000000..a2b7f600a --- /dev/null +++ b/spark/spark-2.2/src/main/scala/hivemall/tools/RegressionDatagen.scala @@ -0,0 +1,67 @@ +/* + * 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 hivemall.tools + +import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.hive.HivemallOps._ +import org.apache.spark.sql.types._ + +object RegressionDatagen { + + /** + * Generate data for regression/classification. + * See [[hivemall.dataset.LogisticRegressionDataGeneratorUDTF]] + * for the details of arguments below. + */ + def exec(sc: SQLContext, + n_partitions: Int = 2, + min_examples: Int = 1000, + n_features: Int = 10, + n_dims: Int = 200, + seed: Int = 43, + dense: Boolean = false, + prob_one: Float = 0.6f, + sort: Boolean = false, + cl: Boolean = false): DataFrame = { + + require(n_partitions > 0, "Non-negative #n_partitions required.") + require(min_examples > 0, "Non-negative #min_examples required.") + require(n_features > 0, "Non-negative #n_features required.") + require(n_dims > 0, "Non-negative #n_dims required.") + + // Calculate #examples to generate in each partition + val n_examples = (min_examples + n_partitions - 1) / n_partitions + + val df = sc.createDataFrame( + sc.sparkContext.parallelize((0 until n_partitions).map(Row(_)), n_partitions), + StructType( + StructField("data", IntegerType, true) :: + Nil) + ) + import sc.implicits._ + df.lr_datagen( + lit(s"-n_examples $n_examples -n_features $n_features -n_dims $n_dims -prob_one $prob_one" + + (if (dense) " -dense" else "") + + (if (sort) " -sort" else "") + + (if (cl) " -cl" else "")) + ).select($"label".cast(DoubleType).as("label"), $"features") + } +} diff --git a/spark/spark-2.2/src/main/scala/org/apache/spark/sql/catalyst/expressions/EachTopK.scala b/spark/spark-2.2/src/main/scala/org/apache/spark/sql/catalyst/expressions/EachTopK.scala new file mode 100644 index 000000000..cac2a5dcd --- /dev/null +++ b/spark/spark-2.2/src/main/scala/org/apache/spark/sql/catalyst/expressions/EachTopK.scala @@ -0,0 +1,133 @@ +/* + * 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.catalyst.expressions + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.catalyst.util.TypeUtils +import org.apache.spark.sql.catalyst.utils.InternalRowPriorityQueue +import org.apache.spark.sql.types._ + +trait TopKHelper { + + def k: Int + def scoreType: DataType + + @transient val ScoreTypes = TypeCollection( + ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType + ) + + protected case class ScoreWriter(writer: UnsafeRowWriter, ordinal: Int) { + + def write(v: Any): Unit = scoreType match { + case ByteType => writer.write(ordinal, v.asInstanceOf[Byte]) + case ShortType => writer.write(ordinal, v.asInstanceOf[Short]) + case IntegerType => writer.write(ordinal, v.asInstanceOf[Int]) + case LongType => writer.write(ordinal, v.asInstanceOf[Long]) + case FloatType => writer.write(ordinal, v.asInstanceOf[Float]) + case DoubleType => writer.write(ordinal, v.asInstanceOf[Double]) + case d: DecimalType => writer.write(ordinal, v.asInstanceOf[Decimal], d.precision, d.scale) + } + } + + protected lazy val scoreOrdering = { + val ordering = TypeUtils.getInterpretedOrdering(scoreType) + if (k > 0) ordering else ordering.reverse + } + + protected lazy val reverseScoreOrdering = scoreOrdering.reverse + + protected lazy val queue: InternalRowPriorityQueue = { + new InternalRowPriorityQueue(Math.abs(k), (x: Any, y: Any) => scoreOrdering.compare(x, y)) + } +} + +case class EachTopK( + k: Int, + scoreExpr: Expression, + groupExprs: Seq[Expression], + elementSchema: StructType, + children: Seq[Attribute]) + extends Generator with TopKHelper with CodegenFallback { + + override val scoreType: DataType = scoreExpr.dataType + + private lazy val groupingProjection: UnsafeProjection = UnsafeProjection.create(groupExprs) + private lazy val scoreProjection: UnsafeProjection = UnsafeProjection.create(scoreExpr :: Nil) + + // The grouping key of the current partition + private var currentGroupingKeys: UnsafeRow = _ + + override def checkInputDataTypes(): TypeCheckResult = { + if (!ScoreTypes.acceptsType(scoreExpr.dataType)) { + TypeCheckResult.TypeCheckFailure(s"$scoreExpr must have a comparable type") + } else { + TypeCheckResult.TypeCheckSuccess + } + } + + private def topKRowsForGroup(): Seq[InternalRow] = if (queue.size > 0) { + val outputRows = queue.iterator.toSeq.reverse + val (headScore, _) = outputRows.head + val rankNum = outputRows.scanLeft((1, headScore)) { case ((rank, prevScore), (score, _)) => + if (prevScore == score) (rank, score) else (rank + 1, score) + } + val topKRow = new UnsafeRow(1) + val bufferHolder = new BufferHolder(topKRow) + val unsafeRowWriter = new UnsafeRowWriter(bufferHolder, 1) + outputRows.zip(rankNum.map(_._1)).map { case ((_, row), index) => + // Writes to an UnsafeRow directly + bufferHolder.reset() + unsafeRowWriter.write(0, index) + topKRow.setTotalSize(bufferHolder.totalSize()) + new JoinedRow(topKRow, row) + } + } else { + Seq.empty + } + + override def eval(input: InternalRow): TraversableOnce[InternalRow] = { + val groupingKeys = groupingProjection(input) + val ret = if (currentGroupingKeys != groupingKeys) { + val topKRows = topKRowsForGroup() + currentGroupingKeys = groupingKeys.copy() + queue.clear() + topKRows + } else { + Iterator.empty + } + queue += Tuple2(scoreProjection(input).get(0, scoreType), input) + ret + } + + override def terminate(): TraversableOnce[InternalRow] = { + if (queue.size > 0) { + val topKRows = topKRowsForGroup() + queue.clear() + topKRows + } else { + Iterator.empty + } + } + + // TODO: Need to support codegen + // protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode +} diff --git a/spark/spark-2.2/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/JoinTopK.scala b/spark/spark-2.2/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/JoinTopK.scala new file mode 100644 index 000000000..556cdc3dd --- /dev/null +++ b/spark/spark-2.2/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/JoinTopK.scala @@ -0,0 +1,68 @@ +/* + * 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.catalyst.plans.logical + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} +import org.apache.spark.sql.types.{BooleanType, IntegerType} + +case class JoinTopK( + k: Int, + left: LogicalPlan, + right: LogicalPlan, + joinType: JoinType, + condition: Option[Expression])( + val scoreExpr: NamedExpression, + private[sql] val rankAttr: Seq[Attribute] = AttributeReference("rank", IntegerType)() :: Nil) + extends BinaryNode with PredicateHelper { + + override def output: Seq[Attribute] = joinType match { + case Inner => rankAttr ++ Seq(scoreExpr.toAttribute) ++ left.output ++ right.output + } + + override def references: AttributeSet = { + AttributeSet((expressions ++ Seq(scoreExpr)).flatMap(_.references)) + } + + override protected def validConstraints: Set[Expression] = joinType match { + case Inner if condition.isDefined => + left.constraints.union(right.constraints) + .union(splitConjunctivePredicates(condition.get).toSet) + } + + override protected final def otherCopyArgs: Seq[AnyRef] = { + scoreExpr :: rankAttr :: Nil + } + + def duplicateResolved: Boolean = left.outputSet.intersect(right.outputSet).isEmpty + + lazy val resolvedExceptNatural: Boolean = { + childrenResolved && + expressions.forall(_.resolved) && + duplicateResolved && + condition.forall(_.dataType == BooleanType) + } + + override lazy val resolved: Boolean = joinType match { + case Inner => resolvedExceptNatural + case tpe => throw new AnalysisException(s"Unsupported using join type $tpe") + } +} diff --git a/spark/spark-2.2/src/main/scala/org/apache/spark/sql/catalyst/utils/InternalRowPriorityQueue.scala b/spark/spark-2.2/src/main/scala/org/apache/spark/sql/catalyst/utils/InternalRowPriorityQueue.scala new file mode 100644 index 000000000..12c20fbbb --- /dev/null +++ b/spark/spark-2.2/src/main/scala/org/apache/spark/sql/catalyst/utils/InternalRowPriorityQueue.scala @@ -0,0 +1,76 @@ +/* + * 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.catalyst.utils + +import java.io.Serializable +import java.util.{PriorityQueue => JPriorityQueue} + +import scala.collection.JavaConverters._ +import scala.collection.generic.Growable + +import org.apache.spark.sql.catalyst.InternalRow + +private[sql] class InternalRowPriorityQueue( + maxSize: Int, + compareFunc: (Any, Any) => Int + ) extends Iterable[(Any, InternalRow)] with Growable[(Any, InternalRow)] with Serializable { + + private[this] val ordering = new Ordering[(Any, InternalRow)] { + override def compare(x: (Any, InternalRow), y: (Any, InternalRow)): Int = + compareFunc(x._1, y._1) + } + + private val underlying = new JPriorityQueue[(Any, InternalRow)](maxSize, ordering) + + override def iterator: Iterator[(Any, InternalRow)] = underlying.iterator.asScala + + override def size: Int = underlying.size + + override def ++=(xs: TraversableOnce[(Any, InternalRow)]): this.type = { + xs.foreach { this += _ } + this + } + + override def +=(elem: (Any, InternalRow)): this.type = { + if (size < maxSize) { + underlying.offer((elem._1, elem._2.copy())) + } else { + maybeReplaceLowest(elem) + } + this + } + + override def +=(elem1: (Any, InternalRow), elem2: (Any, InternalRow), elems: (Any, InternalRow)*) + : this.type = { + this += elem1 += elem2 ++= elems + } + + override def clear() { underlying.clear() } + + private def maybeReplaceLowest(a: (Any, InternalRow)): Boolean = { + val head = underlying.peek() + if (head != null && ordering.gt(a, head)) { + underlying.poll() + underlying.offer((a._1, a._2.copy())) + } else { + false + } + } +} diff --git a/spark/spark-2.2/src/main/scala/org/apache/spark/sql/execution/UserProvidedPlanner.scala b/spark/spark-2.2/src/main/scala/org/apache/spark/sql/execution/UserProvidedPlanner.scala new file mode 100644 index 000000000..09d60a645 --- /dev/null +++ b/spark/spark-2.2/src/main/scala/org/apache/spark/sql/execution/UserProvidedPlanner.scala @@ -0,0 +1,83 @@ +/* + * 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.execution + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.Strategy +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.catalyst.plans.logical.{JoinTopK, LogicalPlan} +import org.apache.spark.sql.internal.SQLConf + +private object ExtractJoinTopKKeys extends Logging with PredicateHelper { + /** (k, scoreExpr, joinType, leftKeys, rightKeys, condition, leftChild, rightChild) */ + type ReturnType = + (Int, NamedExpression, Seq[Attribute], JoinType, Seq[Expression], Seq[Expression], + Option[Expression], LogicalPlan, LogicalPlan) + + def unapply(plan: LogicalPlan): Option[ReturnType] = plan match { + case join @ JoinTopK(k, left, right, joinType, condition) => + logDebug(s"Considering join on: $condition") + val predicates = condition.map(splitConjunctivePredicates).getOrElse(Nil) + val joinKeys = predicates.flatMap { + case EqualTo(l, r) if canEvaluate(l, left) && canEvaluate(r, right) => Some((l, r)) + case EqualTo(l, r) if canEvaluate(l, right) && canEvaluate(r, left) => Some((r, l)) + // Replace null with default value for joining key, then those rows with null in it could + // be joined together + case EqualNullSafe(l, r) if canEvaluate(l, left) && canEvaluate(r, right) => + Some((Coalesce(Seq(l, Literal.default(l.dataType))), + Coalesce(Seq(r, Literal.default(r.dataType))))) + case EqualNullSafe(l, r) if canEvaluate(l, right) && canEvaluate(r, left) => + Some((Coalesce(Seq(r, Literal.default(r.dataType))), + Coalesce(Seq(l, Literal.default(l.dataType))))) + case other => None + } + val otherPredicates = predicates.filterNot { + case EqualTo(l, r) => + canEvaluate(l, left) && canEvaluate(r, right) || + canEvaluate(l, right) && canEvaluate(r, left) + case other => false + } + + if (joinKeys.nonEmpty) { + val (leftKeys, rightKeys) = joinKeys.unzip + logDebug(s"leftKeys:$leftKeys | rightKeys:$rightKeys") + Some((k, join.scoreExpr, join.rankAttr, joinType, leftKeys, rightKeys, + otherPredicates.reduceOption(And), left, right)) + } else { + None + } + + case p => + None + } +} + +private[sql] class UserProvidedPlanner(val conf: SQLConf) extends Strategy { + + override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case ExtractJoinTopKKeys( + k, scoreExpr, rankAttr, _, leftKeys, rightKeys, condition, left, right) => + Seq(joins.ShuffledHashJoinTopKExec( + k, leftKeys, rightKeys, condition, planLater(left), planLater(right))(scoreExpr, rankAttr)) + case _ => + Nil + } +} diff --git a/spark/spark-2.2/src/main/scala/org/apache/spark/sql/execution/datasources/csv/csvExpressions.scala b/spark/spark-2.2/src/main/scala/org/apache/spark/sql/execution/datasources/csv/csvExpressions.scala new file mode 100644 index 000000000..1f56c906e --- /dev/null +++ b/spark/spark-2.2/src/main/scala/org/apache/spark/sql/execution/datasources/csv/csvExpressions.scala @@ -0,0 +1,169 @@ +/* + * 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.execution.datasources.csv + +import com.univocity.parsers.csv.CsvWriter + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, TimeZoneAwareExpression, UnaryExpression} +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +/** + * Converts a csv input string to a [[StructType]] with the specified schema. + * + * TODO: Move this class into org.apache.spark.sql.catalyst.expressions in Spark-v2.2+ + */ +case class CsvToStruct( + schema: StructType, + options: Map[String, String], + child: Expression, + timeZoneId: Option[String] = None) + extends UnaryExpression with TimeZoneAwareExpression with CodegenFallback with ExpectsInputTypes { + + def this(schema: StructType, options: Map[String, String], child: Expression) = + this(schema, options, child, None) + + override def nullable: Boolean = true + + @transient private lazy val csvOptions = new CSVOptions(options, timeZoneId.get) + @transient private lazy val csvParser = new UnivocityParser(schema, schema, csvOptions) + + private def parse(input: String): InternalRow = csvParser.parse(input) + + override def dataType: DataType = schema + + override def nullSafeEval(csv: Any): Any = { + try parse(csv.toString) catch { case _: RuntimeException => null } + } + + override def inputTypes: Seq[AbstractDataType] = StringType :: Nil + + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) +} + +private class CsvGenerator(schema: StructType, options: CSVOptions) { + + // A `ValueConverter` is responsible for converting a value of an `InternalRow` to `String`. + // When the value is null, this converter should not be called. + private type ValueConverter = (InternalRow, Int) => String + + // `ValueConverter`s for all values in the fields of the schema + private val valueConverters: Array[ValueConverter] = + schema.map(_.dataType).map(makeConverter).toArray + + private def makeConverter(dataType: DataType): ValueConverter = dataType match { + case DateType => + (row: InternalRow, ordinal: Int) => + options.dateFormat.format(DateTimeUtils.toJavaDate(row.getInt(ordinal))) + + case TimestampType => + (row: InternalRow, ordinal: Int) => + options.timestampFormat.format(DateTimeUtils.toJavaTimestamp(row.getLong(ordinal))) + + case udt: UserDefinedType[_] => makeConverter(udt.sqlType) + + case dt: DataType => + (row: InternalRow, ordinal: Int) => + row.get(ordinal, dt).toString + } + + def convertRow(row: InternalRow): Seq[String] = { + var i = 0 + val values = new Array[String](row.numFields) + while (i < row.numFields) { + if (!row.isNullAt(i)) { + values(i) = valueConverters(i).apply(row, i) + } else { + values(i) = options.nullValue + } + i += 1 + } + values + } +} + +/** + * Converts a [[StructType]] to a csv output string. + */ +case class StructToCsv( + options: Map[String, String], + child: Expression, + timeZoneId: Option[String] = None) + extends UnaryExpression with TimeZoneAwareExpression with CodegenFallback with ExpectsInputTypes { + override def nullable: Boolean = true + + @transient + private lazy val params = new CSVOptions(options, timeZoneId.get) + + @transient + private lazy val dataSchema = child.dataType.asInstanceOf[StructType] + + @transient + private lazy val writer = new CsvGenerator(dataSchema, params) + + override def dataType: DataType = StringType + + private def verifySchema(schema: StructType): Unit = { + def verifyType(dataType: DataType): Unit = dataType match { + case ByteType | ShortType | IntegerType | LongType | FloatType | + DoubleType | BooleanType | _: DecimalType | TimestampType | + DateType | StringType => + + case udt: UserDefinedType[_] => verifyType(udt.sqlType) + + case _ => + throw new UnsupportedOperationException( + s"CSV data source does not support ${dataType.simpleString} data type.") + } + + schema.foreach(field => verifyType(field.dataType)) + } + + override def checkInputDataTypes(): TypeCheckResult = { + if (StructType.acceptsType(child.dataType)) { + try { + verifySchema(child.dataType.asInstanceOf[StructType]) + TypeCheckResult.TypeCheckSuccess + } catch { + case e: UnsupportedOperationException => + TypeCheckResult.TypeCheckFailure(e.getMessage) + } + } else { + TypeCheckResult.TypeCheckFailure( + s"$prettyName requires that the expression is a struct expression.") + } + } + + override def nullSafeEval(row: Any): Any = { + val rowStr = writer.convertRow(row.asInstanceOf[InternalRow]) + .mkString(params.delimiter.toString) + UTF8String.fromString(rowStr) + } + + override def inputTypes: Seq[AbstractDataType] = StructType :: Nil + + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) +} diff --git a/spark/spark-2.2/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinTopKExec.scala b/spark/spark-2.2/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinTopKExec.scala new file mode 100644 index 000000000..0067bbbc1 --- /dev/null +++ b/spark/spark-2.2/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinTopKExec.scala @@ -0,0 +1,405 @@ +/* + * 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.execution.joins + +import org.apache.spark.TaskContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.utils.InternalRowPriorityQueue +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.metric._ +import org.apache.spark.sql.types._ + +abstract class PriorityQueueShim { + + def insert(score: Any, row: InternalRow): Unit + def get(): Iterator[InternalRow] + def clear(): Unit +} + +case class ShuffledHashJoinTopKExec( + k: Int, + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + condition: Option[Expression], + left: SparkPlan, + right: SparkPlan)( + scoreExpr: NamedExpression, + rankAttr: Seq[Attribute]) + extends BinaryExecNode with TopKHelper with HashJoin with CodegenSupport { + + override lazy val metrics = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) + + override val scoreType: DataType = scoreExpr.dataType + override val joinType: JoinType = Inner + override val buildSide: BuildSide = BuildRight // Only support `BuildRight` + + private lazy val scoreProjection: UnsafeProjection = + UnsafeProjection.create(scoreExpr :: Nil, left.output ++ right.output) + + private lazy val boundCondition = if (condition.isDefined) { + (r: InternalRow) => newPredicate(condition.get, streamedPlan.output ++ buildPlan.output).eval(r) + } else { + (r: InternalRow) => true + } + + private lazy val topKAttr = rankAttr :+ scoreExpr.toAttribute + + private lazy val _priorityQueue = new PriorityQueueShim { + + private val q: InternalRowPriorityQueue = queue + private val joinedRow = new JoinedRow + + override def insert(score: Any, row: InternalRow): Unit = { + q += Tuple2(score, row) + } + + override def get(): Iterator[InternalRow] = { + val outputRows = queue.iterator.toSeq.reverse + val (headScore, _) = outputRows.head + val rankNum = outputRows.scanLeft((1, headScore)) { case ((rank, prevScore), (score, _)) => + if (prevScore == score) (rank, score) else (rank + 1, score) + } + val topKRow = new UnsafeRow(2) + val bufferHolder = new BufferHolder(topKRow) + val unsafeRowWriter = new UnsafeRowWriter(bufferHolder, 2) + val scoreWriter = ScoreWriter(unsafeRowWriter, 1) + outputRows.zip(rankNum.map(_._1)).map { case ((score, row), index) => + // Writes to an UnsafeRow directly + bufferHolder.reset() + unsafeRowWriter.write(0, index) + scoreWriter.write(score) + topKRow.setTotalSize(bufferHolder.totalSize()) + joinedRow.apply(topKRow, row) + }.iterator + } + + override def clear(): Unit = q.clear() + } + + override def output: Seq[Attribute] = joinType match { + case Inner => topKAttr ++ left.output ++ right.output + } + + override protected final def otherCopyArgs: Seq[AnyRef] = { + scoreExpr :: rankAttr :: Nil + } + + override def requiredChildDistribution: Seq[Distribution] = + ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil + + def buildHashedRelation(iter: Iterator[InternalRow]): HashedRelation = { + val context = TaskContext.get() + val relation = HashedRelation(iter, buildKeys, taskMemoryManager = context.taskMemoryManager()) + context.addTaskCompletionListener(_ => relation.close()) + relation + } + + override protected def createResultProjection(): (InternalRow) => InternalRow = joinType match { + case Inner => + // Always put the stream side on left to simplify implementation + // both of left and right side could be null + UnsafeProjection.create( + output, (topKAttr ++ streamedPlan.output ++ buildPlan.output).map(_.withNullability(true))) + } + + protected def InnerJoin( + streamedIter: Iterator[InternalRow], + hashedRelation: HashedRelation, + numOutputRows: SQLMetric): Iterator[InternalRow] = { + val joinRow = new JoinedRow + val joinKeysProj = streamSideKeyGenerator() + val joinedIter = streamedIter.flatMap { srow => + joinRow.withLeft(srow) + val joinKeys = joinKeysProj(srow) // `joinKeys` is also a grouping key + val matches = hashedRelation.get(joinKeys) + if (matches != null) { + matches.map(joinRow.withRight).filter(boundCondition).foreach { resultRow => + _priorityQueue.insert(scoreProjection(resultRow).get(0, scoreType), resultRow) + } + val iter = _priorityQueue.get() + _priorityQueue.clear() + iter + } else { + Seq.empty + } + } + val resultProj = createResultProjection() + (joinedIter ++ queue.iterator.toSeq.sortBy(_._1)(reverseScoreOrdering) + .map(_._2)).map { r => + resultProj(r) + } + } + + override protected def doExecute(): RDD[InternalRow] = { + streamedPlan.execute().zipPartitions(buildPlan.execute()) { (streamIter, buildIter) => + val hashed = buildHashedRelation(buildIter) + InnerJoin(streamIter, hashed, null) + } + } + + override def inputRDDs(): Seq[RDD[InternalRow]] = { + left.execute() :: right.execute() :: Nil + } + + // Accessor for generated code + def priorityQueue(): PriorityQueueShim = _priorityQueue + + /** + * Add a state of HashedRelation and return the variable name for it. + */ + private def prepareHashedRelation(ctx: CodegenContext): String = { + // create a name for HashedRelation + val joinExec = ctx.addReferenceObj("joinExec", this) + val relationTerm = ctx.freshName("relation") + val clsName = HashedRelation.getClass.getName.replace("$", "") + ctx.addMutableState(clsName, relationTerm, + s""" + | $relationTerm = ($clsName) $joinExec.buildHashedRelation(inputs[1]); + | incPeakExecutionMemory($relationTerm.estimatedSize()); + """.stripMargin) + relationTerm + } + + /** + * Creates variables for left part of result row. + * + * In order to defer the access after condition and also only access once in the loop, + * the variables should be declared separately from accessing the columns, we can't use the + * codegen of BoundReference here. + */ + private def createLeftVars(ctx: CodegenContext, leftRow: String): Seq[ExprCode] = { + ctx.INPUT_ROW = leftRow + left.output.zipWithIndex.map { case (a, i) => + val value = ctx.freshName("value") + val valueCode = ctx.getValue(leftRow, a.dataType, i.toString) + // declare it as class member, so we can access the column before or in the loop. + ctx.addMutableState(ctx.javaType(a.dataType), value, "") + if (a.nullable) { + val isNull = ctx.freshName("isNull") + ctx.addMutableState("boolean", isNull, "") + val code = + s""" + |$isNull = $leftRow.isNullAt($i); + |$value = $isNull ? ${ctx.defaultValue(a.dataType)} : ($valueCode); + """.stripMargin + ExprCode(code, isNull, value) + } else { + ExprCode(s"$value = $valueCode;", "false", value) + } + } + } + + /** + * Creates the variables for right part of result row, using BoundReference, since the right + * part are accessed inside the loop. + */ + private def createRightVar(ctx: CodegenContext, rightRow: String): Seq[ExprCode] = { + ctx.INPUT_ROW = rightRow + right.output.zipWithIndex.map { case (a, i) => + BoundReference(i, a.dataType, a.nullable).genCode(ctx) + } + } + + /** + * Returns the code for generating join key for stream side, and expression of whether the key + * has any null in it or not. + */ + private def genStreamSideJoinKey(ctx: CodegenContext, leftRow: String): (ExprCode, String) = { + ctx.INPUT_ROW = leftRow + if (streamedKeys.length == 1 && streamedKeys.head.dataType == LongType) { + // generate the join key as Long + val ev = streamedKeys.head.genCode(ctx) + (ev, ev.isNull) + } else { + // generate the join key as UnsafeRow + val ev = GenerateUnsafeProjection.createCode(ctx, streamedKeys) + (ev, s"${ev.value}.anyNull()") + } + } + + private def createScoreVar(ctx: CodegenContext, row: String): ExprCode = { + ctx.INPUT_ROW = row + BindReferences.bindReference(scoreExpr, left.output ++ right.output).genCode(ctx) + } + + private def createResultVars(ctx: CodegenContext, resultRow: String): Seq[ExprCode] = { + ctx.INPUT_ROW = resultRow + output.zipWithIndex.map { case (a, i) => + val value = ctx.freshName("value") + val valueCode = ctx.getValue(resultRow, a.dataType, i.toString) + // declare it as class member, so we can access the column before or in the loop. + ctx.addMutableState(ctx.javaType(a.dataType), value, "") + if (a.nullable) { + val isNull = ctx.freshName("isNull") + ctx.addMutableState("boolean", isNull, "") + val code = + s""" + |$isNull = $resultRow.isNullAt($i); + |$value = $isNull ? ${ctx.defaultValue(a.dataType)} : ($valueCode); + """.stripMargin + ExprCode(code, isNull, value) + } else { + ExprCode(s"$value = $valueCode;", "false", value) + } + } + } + + /** + * Splits variables based on whether it's used by condition or not, returns the code to create + * these variables before the condition and after the condition. + * + * Only a few columns are used by condition, then we can skip the accessing of those columns + * that are not used by condition also filtered out by condition. + */ + private def splitVarsByCondition( + attributes: Seq[Attribute], + variables: Seq[ExprCode]): (String, String) = { + if (condition.isDefined) { + val condRefs = condition.get.references + val (used, notUsed) = attributes.zip(variables).partition{ case (a, ev) => + condRefs.contains(a) + } + val beforeCond = evaluateVariables(used.map(_._2)) + val afterCond = evaluateVariables(notUsed.map(_._2)) + (beforeCond, afterCond) + } else { + (evaluateVariables(variables), "") + } + } + + override def doProduce(ctx: CodegenContext): String = { + ctx.copyResult = true + + val topKJoin = ctx.addReferenceObj("topKJoin", this) + + // Prepare a priority queue for top-K computing + val pQueue = ctx.freshName("queue") + ctx.addMutableState(classOf[PriorityQueueShim].getName, pQueue, + s"$pQueue = $topKJoin.priorityQueue();") + + // Prepare variables for a left side + val leftIter = ctx.freshName("leftIter") + ctx.addMutableState("scala.collection.Iterator", leftIter, s"$leftIter = inputs[0];") + val leftRow = ctx.freshName("leftRow") + ctx.addMutableState("InternalRow", leftRow, "") + val leftVars = createLeftVars(ctx, leftRow) + + // Prepare variables for a right side + val rightRow = ctx.freshName("rightRow") + val rightVars = createRightVar(ctx, rightRow) + + // Build a hashed relation from a right side + val buildRelation = prepareHashedRelation(ctx) + + // Project join keys from a left side + val (keyEv, anyNull) = genStreamSideJoinKey(ctx, leftRow) + + // Prepare variables for joined rows + val joinedRow = ctx.freshName("joinedRow") + val joinedRowCls = classOf[JoinedRow].getName + ctx.addMutableState(joinedRowCls, joinedRow, s"$joinedRow = new $joinedRowCls();") + + // Project score values from joined rows + val scoreVar = createScoreVar(ctx, joinedRow) + + // Prepare variables for output rows + val resultRow = ctx.freshName("resultRow") + val resultVars = createResultVars(ctx, resultRow) + + val (beforeLoop, condCheck) = if (condition.isDefined) { + // Split the code of creating variables based on whether it's used by condition or not. + val loaded = ctx.freshName("loaded") + val (leftBefore, leftAfter) = splitVarsByCondition(left.output, leftVars) + val (rightBefore, rightAfter) = splitVarsByCondition(right.output, rightVars) + // Generate code for condition + ctx.currentVars = leftVars ++ rightVars + val cond = BindReferences.bindReference(condition.get, output).genCode(ctx) + // evaluate the columns those used by condition before loop + val before = s""" + |boolean $loaded = false; + |$leftBefore + """.stripMargin + + val checking = s""" + |$rightBefore + |${cond.code} + |if (${cond.isNull} || !${cond.value}) continue; + |if (!$loaded) { + | $loaded = true; + | $leftAfter + |} + |$rightAfter + """.stripMargin + (before, checking) + } else { + (evaluateVariables(leftVars), "") + } + + val numOutput = metricTerm(ctx, "numOutputRows") + + val matches = ctx.freshName("matches") + val topKRows = ctx.freshName("topKRows") + val iteratorCls = classOf[Iterator[UnsafeRow]].getName + + s""" + |$leftRow = null; + |while ($leftIter.hasNext()) { + | $leftRow = (InternalRow) $leftIter.next(); + | + | // Generate join key for stream side + | ${keyEv.code} + | + | // Find matches from HashedRelation + | $iteratorCls $matches = $anyNull? null : ($iteratorCls)$buildRelation.get(${keyEv.value}); + | if ($matches == null) continue; + | + | // Join top-K right rows + | while ($matches.hasNext()) { + | ${beforeLoop.trim} + | InternalRow $rightRow = (InternalRow) $matches.next(); + | ${condCheck.trim} + | InternalRow row = $joinedRow.apply($leftRow, $rightRow); + | // Compute a score for the `row` + | ${scoreVar.code} + | $pQueue.insert(${scoreVar.value}, row); + | } + | + | // Get top-K rows + | $iteratorCls $topKRows = $pQueue.get(); + | $pQueue.clear(); + | + | // Output top-K rows + | while ($topKRows.hasNext()) { + | InternalRow $resultRow = (InternalRow) $topKRows.next(); + | $numOutput.add(1); + | ${consume(ctx, resultVars)} + | } + | + | if (shouldStop()) return; + |} + """.stripMargin + } +} diff --git a/spark/spark-2.2/src/main/scala/org/apache/spark/sql/hive/HivemallGroupedDataset.scala b/spark/spark-2.2/src/main/scala/org/apache/spark/sql/hive/HivemallGroupedDataset.scala new file mode 100644 index 000000000..75f4bfce3 --- /dev/null +++ b/spark/spark-2.2/src/main/scala/org/apache/spark/sql/hive/HivemallGroupedDataset.scala @@ -0,0 +1,304 @@ +/* + * 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 org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.Dataset +import org.apache.spark.sql.RelationalGroupedDataset +import org.apache.spark.sql.catalyst.analysis.UnresolvedAlias +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.Aggregate +import org.apache.spark.sql.catalyst.plans.logical.Pivot +import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper +import org.apache.spark.sql.types._ + +/** + * Groups the [[DataFrame]] using the specified columns, so we can run aggregation on them. + * + * @groupname ensemble + * @groupname ftvec.trans + * @groupname evaluation + */ +final class HivemallGroupedDataset(groupBy: RelationalGroupedDataset) { + + /** + * @see hivemall.ensemble.bagging.VotedAvgUDAF + * @group ensemble + */ + def voted_avg(weight: String): DataFrame = { + // checkType(weight, NumericType) + val udaf = HiveUDAFFunction( + "voted_avg", + new HiveFunctionWrapper("hivemall.ensemble.bagging.WeightVotedAvgUDAF"), + Seq(weight).map(df.col(_).expr), + isUDAFBridgeRequired = true) + .toAggregateExpression() + toDF((Alias(udaf, udaf.prettyName)() :: Nil).toSeq) + } + + /** + * @see hivemall.ensemble.bagging.WeightVotedAvgUDAF + * @group ensemble + */ + def weight_voted_avg(weight: String): DataFrame = { + // checkType(weight, NumericType) + val udaf = HiveUDAFFunction( + "weight_voted_avg", + new HiveFunctionWrapper("hivemall.ensemble.bagging.WeightVotedAvgUDAF"), + Seq(weight).map(df.col(_).expr), + isUDAFBridgeRequired = true) + .toAggregateExpression() + toDF((Alias(udaf, udaf.prettyName)() :: Nil).toSeq) + } + + /** + * @see hivemall.ensemble.ArgminKLDistanceUDAF + * @group ensemble + */ + def argmin_kld(weight: String, conv: String): DataFrame = { + // checkType(weight, NumericType) + // checkType(conv, NumericType) + val udaf = HiveUDAFFunction( + "argmin_kld", + new HiveFunctionWrapper("hivemall.ensemble.ArgminKLDistanceUDAF"), + Seq(weight, conv).map(df.col(_).expr), + isUDAFBridgeRequired = true) + .toAggregateExpression() + toDF((Alias(udaf, udaf.prettyName)() :: Nil).toSeq) + } + + /** + * @see hivemall.ensemble.MaxValueLabelUDAF" + * @group ensemble + */ + def max_label(score: String, label: String): DataFrame = { + // checkType(score, NumericType) + checkType(label, StringType) + val udaf = HiveUDAFFunction( + "max_label", + new HiveFunctionWrapper("hivemall.ensemble.MaxValueLabelUDAF"), + Seq(score, label).map(df.col(_).expr), + isUDAFBridgeRequired = true) + .toAggregateExpression() + toDF((Alias(udaf, udaf.prettyName)() :: Nil).toSeq) + } + + /** + * @see hivemall.ensemble.MaxRowUDAF + * @group ensemble + */ + def maxrow(score: String, label: String): DataFrame = { + // checkType(score, NumericType) + checkType(label, StringType) + val udaf = HiveUDAFFunction( + "maxrow", + new HiveFunctionWrapper("hivemall.ensemble.MaxRowUDAF"), + Seq(score, label).map(df.col(_).expr), + isUDAFBridgeRequired = false) + .toAggregateExpression() + toDF((Alias(udaf, udaf.prettyName)() :: Nil).toSeq) + } + + /** + * @see hivemall.smile.tools.RandomForestEnsembleUDAF + * @group ensemble + */ + def rf_ensemble(predict: String): DataFrame = { + // checkType(predict, NumericType) + val udaf = HiveUDAFFunction( + "rf_ensemble", + new HiveFunctionWrapper("hivemall.smile.tools.RandomForestEnsembleUDAF"), + Seq(predict).map(df.col(_).expr), + isUDAFBridgeRequired = false) + .toAggregateExpression() + toDF((Alias(udaf, udaf.prettyName)() :: Nil).toSeq) + } + + /** + * @see hivemall.tools.matrix.TransposeAndDotUDAF + */ + def transpose_and_dot(X: String, Y: String): DataFrame = { + val udaf = HiveUDAFFunction( + "transpose_and_dot", + new HiveFunctionWrapper("hivemall.tools.matrix.TransposeAndDotUDAF"), + Seq(X, Y).map(df.col(_).expr), + isUDAFBridgeRequired = false) + .toAggregateExpression() + toDF(Seq(Alias(udaf, udaf.prettyName)())) + } + + /** + * @see hivemall.ftvec.trans.OnehotEncodingUDAF + * @group ftvec.trans + */ + def onehot_encoding(cols: String*): DataFrame = { + val udaf = HiveUDAFFunction( + "onehot_encoding", + new HiveFunctionWrapper("hivemall.ftvec.trans.OnehotEncodingUDAF"), + cols.map(df.col(_).expr), + isUDAFBridgeRequired = false) + .toAggregateExpression() + toDF(Seq(Alias(udaf, udaf.prettyName)())) + } + + /** + * @see hivemall.ftvec.selection.SignalNoiseRatioUDAF + */ + def snr(X: String, Y: String): DataFrame = { + val udaf = HiveUDAFFunction( + "snr", + new HiveFunctionWrapper("hivemall.ftvec.selection.SignalNoiseRatioUDAF"), + Seq(X, Y).map(df.col(_).expr), + isUDAFBridgeRequired = false) + .toAggregateExpression() + toDF(Seq(Alias(udaf, udaf.prettyName)())) + } + + /** + * @see hivemall.evaluation.MeanAbsoluteErrorUDAF + * @group evaluation + */ + def mae(predict: String, target: String): DataFrame = { + checkType(predict, FloatType) + checkType(target, FloatType) + val udaf = HiveUDAFFunction( + "mae", + new HiveFunctionWrapper("hivemall.evaluation.MeanAbsoluteErrorUDAF"), + Seq(predict, target).map(df.col(_).expr), + isUDAFBridgeRequired = true) + .toAggregateExpression() + toDF((Alias(udaf, udaf.prettyName)() :: Nil).toSeq) + } + + /** + * @see hivemall.evaluation.MeanSquareErrorUDAF + * @group evaluation + */ + def mse(predict: String, target: String): DataFrame = { + checkType(predict, FloatType) + checkType(target, FloatType) + val udaf = HiveUDAFFunction( + "mse", + new HiveFunctionWrapper("hivemall.evaluation.MeanSquaredErrorUDAF"), + Seq(predict, target).map(df.col(_).expr), + isUDAFBridgeRequired = true) + .toAggregateExpression() + toDF((Alias(udaf, udaf.prettyName)() :: Nil).toSeq) + } + + /** + * @see hivemall.evaluation.RootMeanSquareErrorUDAF + * @group evaluation + */ + def rmse(predict: String, target: String): DataFrame = { + checkType(predict, FloatType) + checkType(target, FloatType) + val udaf = HiveUDAFFunction( + "rmse", + new HiveFunctionWrapper("hivemall.evaluation.RootMeanSquaredErrorUDAF"), + Seq(predict, target).map(df.col(_).expr), + isUDAFBridgeRequired = true) + .toAggregateExpression() + toDF((Alias(udaf, udaf.prettyName)() :: Nil).toSeq) + } + + /** + * @see hivemall.evaluation.FMeasureUDAF + * @group evaluation + */ + def f1score(predict: String, target: String): DataFrame = { + // checkType(target, ArrayType(IntegerType)) + // checkType(predict, ArrayType(IntegerType)) + val udaf = HiveUDAFFunction( + "f1score", + new HiveFunctionWrapper("hivemall.evaluation.FMeasureUDAF"), + Seq(predict, target).map(df.col(_).expr), + isUDAFBridgeRequired = true) + .toAggregateExpression() + toDF((Alias(udaf, udaf.prettyName)() :: Nil).toSeq) + } + + /** + * [[RelationalGroupedDataset]] has the three values as private fields, so, to inject Hivemall + * aggregate functions, we fetch them via Java Reflections. + */ + private val df = getPrivateField[DataFrame]("org$apache$spark$sql$RelationalGroupedDataset$$df") + private val groupingExprs = getPrivateField[Seq[Expression]]("groupingExprs") + private val groupType = getPrivateField[RelationalGroupedDataset.GroupType]("groupType") + + private def getPrivateField[T](name: String): T = { + val field = groupBy.getClass.getDeclaredField(name) + field.setAccessible(true) + field.get(groupBy).asInstanceOf[T] + } + + private def toDF(aggExprs: Seq[Expression]): DataFrame = { + val aggregates = if (df.sqlContext.conf.dataFrameRetainGroupColumns) { + groupingExprs ++ aggExprs + } else { + aggExprs + } + + val aliasedAgg = aggregates.map(alias) + + groupType match { + case RelationalGroupedDataset.GroupByType => + Dataset.ofRows( + df.sparkSession, Aggregate(groupingExprs, aliasedAgg, df.logicalPlan)) + case RelationalGroupedDataset.RollupType => + Dataset.ofRows( + df.sparkSession, Aggregate(Seq(Rollup(groupingExprs)), aliasedAgg, df.logicalPlan)) + case RelationalGroupedDataset.CubeType => + Dataset.ofRows( + df.sparkSession, Aggregate(Seq(Cube(groupingExprs)), aliasedAgg, df.logicalPlan)) + case RelationalGroupedDataset.PivotType(pivotCol, values) => + val aliasedGrps = groupingExprs.map(alias) + Dataset.ofRows( + df.sparkSession, Pivot(aliasedGrps, pivotCol, values, aggExprs, df.logicalPlan)) + } + } + + private def alias(expr: Expression): NamedExpression = expr match { + case u: UnresolvedAttribute => UnresolvedAlias(u) + case expr: NamedExpression => expr + case expr: Expression => Alias(expr, expr.prettyName)() + } + + private def checkType(colName: String, expected: DataType) = { + val dataType = df.resolve(colName).dataType + if (dataType != expected) { + throw new AnalysisException( + s""""$colName" must be $expected, however it is $dataType""") + } + } +} + +object HivemallGroupedDataset { + + /** + * Implicitly inject the [[HivemallGroupedDataset]] into [[RelationalGroupedDataset]]. + */ + implicit def relationalGroupedDatasetToHivemallOne( + groupBy: RelationalGroupedDataset): HivemallGroupedDataset = { + new HivemallGroupedDataset(groupBy) + } +} diff --git a/spark/spark-2.2/src/main/scala/org/apache/spark/sql/hive/HivemallOps.scala b/spark/spark-2.2/src/main/scala/org/apache/spark/sql/hive/HivemallOps.scala new file mode 100644 index 000000000..22d3153b2 --- /dev/null +++ b/spark/spark-2.2/src/main/scala/org/apache/spark/sql/hive/HivemallOps.scala @@ -0,0 +1,1538 @@ +/* + * 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.util.UUID + +import org.apache.spark.annotation.Experimental +import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.HivemallFeature +import org.apache.spark.ml.linalg.{DenseVector, SparseVector, VectorUDT} +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.encoders.RowEncoder +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.Inner +import org.apache.spark.sql.catalyst.plans.logical.{Generate, JoinTopK, LogicalPlan} +import org.apache.spark.sql.execution.UserProvidedPlanner +import org.apache.spark.sql.execution.datasources.csv.{CsvToStruct, StructToCsv} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +/** + * Hivemall wrapper and some utility functions for DataFrame. + * + * @groupname regression + * @groupname classifier + * @groupname classifier.multiclass + * @groupname xgboost + * @groupname anomaly + * @groupname knn.similarity + * @groupname knn.distance + * @groupname knn.lsh + * @groupname ftvec + * @groupname ftvec.amplify + * @groupname ftvec.hashing + * @groupname ftvec.scaling + * @groupname ftvec.conv + * @groupname ftvec.trans + * @groupname misc + */ +final class HivemallOps(df: DataFrame) extends Logging { + import internal.HivemallOpsImpl._ + + private[this] lazy val _sparkSession = df.sparkSession + private[this] lazy val _analyzer = _sparkSession.sessionState.analyzer + private[this] lazy val _strategy = new UserProvidedPlanner(_sparkSession.sqlContext.conf) + + /** + * @see [[hivemall.regression.AdaDeltaUDTF]] + * @group regression + */ + @scala.annotation.varargs + def train_adadelta(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.regression.AdaDeltaUDTF", + "train_adadelta", + setMixServs(toHivemallFeatures(exprs)), + Seq("feature", "weight") + ) + } + + /** + * @see [[hivemall.regression.AdaGradUDTF]] + * @group regression + */ + @scala.annotation.varargs + def train_adagrad(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.regression.AdaGradUDTF", + "train_adagrad", + setMixServs(toHivemallFeatures(exprs)), + Seq("feature", "weight") + ) + } + + /** + * @see [[hivemall.regression.AROWRegressionUDTF]] + * @group regression + */ + @scala.annotation.varargs + def train_arow_regr(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.regression.AROWRegressionUDTF", + "train_arow_regr", + setMixServs(toHivemallFeatures(exprs)), + Seq("feature", "weight", "conv") + ) + } + + /** + * @see [[hivemall.regression.AROWRegressionUDTF.AROWe]] + * @group regression + */ + @scala.annotation.varargs + def train_arowe_regr(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.regression.AROWRegressionUDTF$AROWe", + "train_arowe_regr", + setMixServs(toHivemallFeatures(exprs)), + Seq("feature", "weight", "conv") + ) + } + + /** + * @see [[hivemall.regression.AROWRegressionUDTF.AROWe2]] + * @group regression + */ + @scala.annotation.varargs + def train_arowe2_regr(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.regression.AROWRegressionUDTF$AROWe2", + "train_arowe2_regr", + setMixServs(toHivemallFeatures(exprs)), + Seq("feature", "weight", "conv") + ) + } + + /** + * @see [[hivemall.regression.LogressUDTF]] + * @group regression + */ + @scala.annotation.varargs + def train_logregr(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.regression.LogressUDTF", + "train_logregr", + setMixServs(toHivemallFeatures(exprs)), + Seq("feature", "weight") + ) + } + + /** + * @see [[hivemall.regression.PassiveAggressiveRegressionUDTF]] + * @group regression + */ + @scala.annotation.varargs + def train_pa1_regr(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.regression.PassiveAggressiveRegressionUDTF", + "train_pa1_regr", + setMixServs(toHivemallFeatures(exprs)), + Seq("feature", "weight") + ) + } + + /** + * @see [[hivemall.regression.PassiveAggressiveRegressionUDTF.PA1a]] + * @group regression + */ + @scala.annotation.varargs + def train_pa1a_regr(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.regression.PassiveAggressiveRegressionUDTF$PA1a", + "train_pa1a_regr", + setMixServs(toHivemallFeatures(exprs)), + Seq("feature", "weight") + ) + } + + /** + * @see [[hivemall.regression.PassiveAggressiveRegressionUDTF.PA2]] + * @group regression + */ + @scala.annotation.varargs + def train_pa2_regr(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.regression.PassiveAggressiveRegressionUDTF$PA2", + "train_pa2_regr", + setMixServs(toHivemallFeatures(exprs)), + Seq("feature", "weight") + ) + } + + /** + * @see [[hivemall.regression.PassiveAggressiveRegressionUDTF.PA2a]] + * @group regression + */ + @scala.annotation.varargs + def train_pa2a_regr(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.regression.PassiveAggressiveRegressionUDTF$PA2a", + "train_pa2a_regr", + setMixServs(toHivemallFeatures(exprs)), + Seq("feature", "weight") + ) + } + + /** + * @see [[hivemall.smile.regression.RandomForestRegressionUDTF]] + * @group regression + */ + @scala.annotation.varargs + def train_randomforest_regr(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.smile.regression.RandomForestRegressionUDTF", + "train_randomforest_regr", + setMixServs(toHivemallFeatures(exprs)), + Seq("model_id", "model_type", "pred_model", "var_importance", "oob_errors", "oob_tests") + ) + } + + /** + * @see [[hivemall.classifier.PerceptronUDTF]] + * @group classifier + */ + @scala.annotation.varargs + def train_perceptron(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.classifier.PerceptronUDTF", + "train_perceptron", + setMixServs(toHivemallFeatures(exprs)), + Seq("feature", "weight") + ) + } + + /** + * @see [[hivemall.classifier.PassiveAggressiveUDTF]] + * @group classifier + */ + @scala.annotation.varargs + def train_pa(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.classifier.PassiveAggressiveUDTF", + "train_pa", + setMixServs(toHivemallFeatures(exprs)), + Seq("feature", "weight") + ) + } + + /** + * @see [[hivemall.classifier.PassiveAggressiveUDTF.PA1]] + * @group classifier + */ + @scala.annotation.varargs + def train_pa1(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.classifier.PassiveAggressiveUDTF$PA1", + "train_pa1", + setMixServs(toHivemallFeatures(exprs)), + Seq("feature", "weight") + ) + } + + /** + * @see [[hivemall.classifier.PassiveAggressiveUDTF.PA2]] + * @group classifier + */ + @scala.annotation.varargs + def train_pa2(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.classifier.PassiveAggressiveUDTF$PA2", + "train_pa2", + setMixServs(toHivemallFeatures(exprs)), + Seq("feature", "weight") + ) + } + + /** + * @see [[hivemall.classifier.ConfidenceWeightedUDTF]] + * @group classifier + */ + @scala.annotation.varargs + def train_cw(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.classifier.ConfidenceWeightedUDTF", + "train_cw", + setMixServs(toHivemallFeatures(exprs)), + Seq("feature", "weight", "conv") + ) + } + + /** + * @see [[hivemall.classifier.AROWClassifierUDTF]] + * @group classifier + */ + @scala.annotation.varargs + def train_arow(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.classifier.AROWClassifierUDTF", + "train_arow", + setMixServs(toHivemallFeatures(exprs)), + Seq("feature", "weight", "conv") + ) + } + + /** + * @see [[hivemall.classifier.AROWClassifierUDTF.AROWh]] + * @group classifier + */ + @scala.annotation.varargs + def train_arowh(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.classifier.AROWClassifierUDTF$AROWh", + "train_arowh", + setMixServs(toHivemallFeatures(exprs)), + Seq("feature", "weight", "conv") + ) + } + + /** + * @see [[hivemall.classifier.SoftConfideceWeightedUDTF.SCW1]] + * @group classifier + */ + @scala.annotation.varargs + def train_scw(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.classifier.SoftConfideceWeightedUDTF$SCW1", + "train_scw", + setMixServs(toHivemallFeatures(exprs)), + Seq("feature", "weight", "conv") + ) + } + + /** + * @see [[hivemall.classifier.SoftConfideceWeightedUDTF.SCW1]] + * @group classifier + */ + @scala.annotation.varargs + def train_scw2(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.classifier.SoftConfideceWeightedUDTF$SCW2", + "train_scw2", + setMixServs(toHivemallFeatures(exprs)), + Seq("feature", "weight", "conv") + ) + } + + /** + * @see [[hivemall.classifier.AdaGradRDAUDTF]] + * @group classifier + */ + @scala.annotation.varargs + def train_adagrad_rda(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.classifier.AdaGradRDAUDTF", + "train_adagrad_rda", + setMixServs(toHivemallFeatures(exprs)), + Seq("feature", "weight") + ) + } + + /** + * @see [[hivemall.smile.classification.RandomForestClassifierUDTF]] + * @group classifier + */ + @scala.annotation.varargs + def train_randomforest_classifier(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.smile.classification.RandomForestClassifierUDTF", + "train_randomforest_classifier", + setMixServs(toHivemallFeatures(exprs)), + Seq("model_id", "model_type", "pred_model", "var_importance", "oob_errors", "oob_tests") + ) + } + + /** + * @see [[hivemall.classifier.multiclass.MulticlassPerceptronUDTF]] + * @group classifier.multiclass + */ + @scala.annotation.varargs + def train_multiclass_perceptron(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.classifier.multiclass.MulticlassPerceptronUDTF", + "train_multiclass_perceptron", + setMixServs(toHivemallFeatures(exprs)), + Seq("label", "feature", "weight") + ) + } + + /** + * @see [[hivemall.classifier.multiclass.MulticlassPassiveAggressiveUDTF]] + * @group classifier.multiclass + */ + @scala.annotation.varargs + def train_multiclass_pa(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.classifier.multiclass.MulticlassPassiveAggressiveUDTF", + "train_multiclass_pa", + setMixServs(toHivemallFeatures(exprs)), + Seq("label", "feature", "weight") + ) + } + + /** + * @see [[hivemall.classifier.multiclass.MulticlassPassiveAggressiveUDTF.PA1]] + * @group classifier.multiclass + */ + @scala.annotation.varargs + def train_multiclass_pa1(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.classifier.multiclass.MulticlassPassiveAggressiveUDTF$PA1", + "train_multiclass_pa1", + setMixServs(toHivemallFeatures(exprs)), + Seq("label", "feature", "weight") + ) + } + + /** + * @see [[hivemall.classifier.multiclass.MulticlassPassiveAggressiveUDTF.PA2]] + * @group classifier.multiclass + */ + @scala.annotation.varargs + def train_multiclass_pa2(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.classifier.multiclass.MulticlassPassiveAggressiveUDTF$PA2", + "train_multiclass_pa2", + setMixServs(toHivemallFeatures(exprs)), + Seq("label", "feature", "weight") + ) + } + + /** + * @see [[hivemall.classifier.multiclass.MulticlassConfidenceWeightedUDTF]] + * @group classifier.multiclass + */ + @scala.annotation.varargs + def train_multiclass_cw(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.classifier.multiclass.MulticlassConfidenceWeightedUDTF", + "train_multiclass_cw", + setMixServs(toHivemallFeatures(exprs)), + Seq("label", "feature", "weight", "conv") + ) + } + + /** + * @see [[hivemall.classifier.multiclass.MulticlassAROWClassifierUDTF]] + * @group classifier.multiclass + */ + @scala.annotation.varargs + def train_multiclass_arow(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.classifier.multiclass.MulticlassAROWClassifierUDTF", + "train_multiclass_arow", + setMixServs(toHivemallFeatures(exprs)), + Seq("label", "feature", "weight", "conv") + ) + } + + /** + * @see [[hivemall.classifier.multiclass.MulticlassSoftConfidenceWeightedUDTF.SCW1]] + * @group classifier.multiclass + */ + @scala.annotation.varargs + def train_multiclass_scw(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.classifier.multiclass.MulticlassSoftConfidenceWeightedUDTF$SCW1", + "train_multiclass_scw", + setMixServs(toHivemallFeatures(exprs)), + Seq("label", "feature", "weight", "conv") + ) + } + + /** + * @see [[hivemall.classifier.multiclass.MulticlassSoftConfidenceWeightedUDTF.SCW2]] + * @group classifier.multiclass + */ + @scala.annotation.varargs + def train_multiclass_scw2(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.classifier.multiclass.MulticlassSoftConfidenceWeightedUDTF$SCW2", + "train_multiclass_scw2", + setMixServs(toHivemallFeatures(exprs)), + Seq("label", "feature", "weight", "conv") + ) + } + + /** + * :: Experimental :: + * @see [[hivemall.xgboost.regression.XGBoostRegressionUDTF]] + * @group xgboost + */ + @Experimental + @scala.annotation.varargs + def train_xgboost_regr(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.xgboost.regression.XGBoostRegressionUDTF", + "train_xgboost_regr", + setMixServs(toHivemallFeatures(exprs)), + Seq("model_id", "pred_model") + ) + } + + /** + * :: Experimental :: + * @see [[hivemall.xgboost.classification.XGBoostBinaryClassifierUDTF]] + * @group xgboost + */ + @Experimental + @scala.annotation.varargs + def train_xgboost_classifier(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.xgboost.classification.XGBoostBinaryClassifierUDTF", + "train_xgboost_classifier", + setMixServs(toHivemallFeatures(exprs)), + Seq("model_id", "pred_model") + ) + } + + /** + * :: Experimental :: + * @see [[hivemall.xgboost.classification.XGBoostMulticlassClassifierUDTF]] + * @group xgboost + */ + @Experimental + @scala.annotation.varargs + def train_xgboost_multiclass_classifier(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.xgboost.classification.XGBoostMulticlassClassifierUDTF", + "train_xgboost_multiclass_classifier", + setMixServs(toHivemallFeatures(exprs)), + Seq("model_id", "pred_model") + ) + } + + /** + * :: Experimental :: + * @see [[hivemall.xgboost.tools.XGBoostPredictUDTF]] + * @group xgboost + */ + @Experimental + @scala.annotation.varargs + def xgboost_predict(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.xgboost.tools.XGBoostPredictUDTF", + "xgboost_predict", + setMixServs(toHivemallFeatures(exprs)), + Seq("rowid", "predicted") + ) + } + + /** + * :: Experimental :: + * @see [[hivemall.xgboost.tools.XGBoostMulticlassPredictUDTF]] + * @group xgboost + */ + @Experimental + @scala.annotation.varargs + def xgboost_multiclass_predict(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.xgboost.tools.XGBoostMulticlassPredictUDTF", + "xgboost_multiclass_predict", + setMixServs(toHivemallFeatures(exprs)), + Seq("rowid", "label", "probability") + ) + } + + /** + * @see [[hivemall.knn.lsh.MinHashUDTF]] + * @group knn.lsh + */ + @scala.annotation.varargs + def minhash(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.knn.lsh.MinHashUDTF", + "minhash", + setMixServs(toHivemallFeatures(exprs)), + Seq("clusterid", "item") + ) + } + + /** + * @see [[hivemall.ftvec.amplify.AmplifierUDTF]] + * @group ftvec.amplify + */ + @scala.annotation.varargs + def amplify(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.ftvec.amplify.AmplifierUDTF", + "amplify", + setMixServs(toHivemallFeatures(exprs)), + Seq("clusterid", "item") + ) + } + + /** + * @see [[hivemall.ftvec.amplify.RandomAmplifierUDTF]] + * @group ftvec.amplify + */ + @scala.annotation.varargs + def rand_amplify(exprs: Column*): DataFrame = withTypedPlan { + throw new UnsupportedOperationException("`rand_amplify` not supported yet") + } + + /** + * Amplifies and shuffle data inside partitions. + * @group ftvec.amplify + */ + def part_amplify(xtimes: Column): DataFrame = { + val xtimesInt = xtimes.expr match { + case Literal(v: Any, IntegerType) => v.asInstanceOf[Int] + case e => throw new AnalysisException("`xtimes` must be integer, however " + e) + } + val rdd = df.rdd.mapPartitions({ iter => + val elems = iter.flatMap{ row => + Seq.fill[Row](xtimesInt)(row) + } + // Need to check how this shuffling affects results + scala.util.Random.shuffle(elems) + }, true) + df.sqlContext.createDataFrame(rdd, df.schema) + } + + /** + * Quantifies input columns. + * @see [[hivemall.ftvec.conv.QuantifyColumnsUDTF]] + * @group ftvec.conv + */ + @scala.annotation.varargs + def quantify(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.ftvec.conv.QuantifyColumnsUDTF", + "quantify", + setMixServs(toHivemallFeatures(exprs)), + (0 until exprs.size - 1).map(i => s"c$i") + ) + } + + /** + * @see [[hivemall.ftvec.trans.BinarizeLabelUDTF]] + * @group ftvec.trans + */ + @scala.annotation.varargs + def binarize_label(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.ftvec.trans.BinarizeLabelUDTF", + "binarize_label", + setMixServs(toHivemallFeatures(exprs)), + (0 until exprs.size - 1).map(i => s"c$i") + ) + } + + /** + * @see [[hivemall.ftvec.trans.QuantifiedFeaturesUDTF]] + * @group ftvec.trans + */ + @scala.annotation.varargs + def quantified_features(exprs: Column*): DataFrame = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.ftvec.trans.QuantifiedFeaturesUDTF", + "quantified_features", + setMixServs(toHivemallFeatures(exprs)), + Seq("features") + ) + } + + /** + * Splits Seq[String] into pieces. + * @group ftvec + */ + def explode_array(expr: Column): DataFrame = { + df.explode(expr) { case Row(v: Seq[_]) => + // Type erasure removes the component type in Seq + v.map(s => HivemallFeature(s.asInstanceOf[String])) + } + } + + /** + * Splits [[Vector]] into pieces. + * @group ftvec + */ + def explode_vector(expr: Column): DataFrame = { + val elementSchema = StructType( + StructField("feature", StringType) :: StructField("weight", DoubleType) :: Nil) + val explodeFunc: Row => TraversableOnce[InternalRow] = (row: Row) => { + row.get(0) match { + case dv: DenseVector => + dv.values.zipWithIndex.map { + case (value, index) => + InternalRow(UTF8String.fromString(s"$index"), value) + } + case sv: SparseVector => + sv.values.zip(sv.indices).map { + case (value, index) => + InternalRow(UTF8String.fromString(s"$index"), value) + } + } + } + withTypedPlan { + Generate( + UserDefinedGenerator(elementSchema, explodeFunc, expr.expr :: Nil), + join = true, outer = false, None, + generatorOutput = Nil, + df.logicalPlan) + } + } + + /** + * Returns `top-k` records for each `group`. + * @group misc + */ + def each_top_k(k: Column, score: Column, group: Column*): DataFrame = withTypedPlan { + val kInt = k.expr match { + case Literal(v: Any, IntegerType) => v.asInstanceOf[Int] + case e => throw new AnalysisException("`k` must be integer, however " + e) + } + if (kInt == 0) { + throw new AnalysisException("`k` must not have 0") + } + val clusterDf = df.repartition(group: _*).sortWithinPartitions(group: _*) + .select(score, Column("*")) + val analyzedPlan = clusterDf.queryExecution.analyzed + val inputAttrs = analyzedPlan.output + val scoreExpr = BindReferences.bindReference(analyzedPlan.expressions.head, inputAttrs) + val groupNames = group.map { _.expr match { + case ne: NamedExpression => ne.name + case ua: UnresolvedAttribute => ua.name + }} + val groupExprs = analyzedPlan.expressions.filter { + case ne: NamedExpression => groupNames.contains(ne.name) + }.map { e => + BindReferences.bindReference(e, inputAttrs) + } + val rankField = StructField("rank", IntegerType) + Generate( + generator = EachTopK( + k = kInt, + scoreExpr = scoreExpr, + groupExprs = groupExprs, + elementSchema = StructType( + rankField +: inputAttrs.map(d => StructField(d.name, d.dataType)) + ), + children = inputAttrs + ), + join = false, + outer = false, + qualifier = None, + generatorOutput = Seq(rankField.name).map(UnresolvedAttribute(_)) ++ inputAttrs, + child = analyzedPlan + ) + } + + /** + * :: Experimental :: + * Joins input two tables with the given keys and the top-k highest `score` values. + * @group misc + */ + @Experimental + def top_k_join(k: Column, right: DataFrame, joinExprs: Column, score: Column) + : DataFrame = withTypedPlanInCustomStrategy { + val kInt = k.expr match { + case Literal(v: Any, IntegerType) => v.asInstanceOf[Int] + case e => throw new AnalysisException("`k` must be integer, however " + e) + } + if (kInt == 0) { + throw new AnalysisException("`k` must not have 0") + } + JoinTopK(kInt, df.logicalPlan, right.logicalPlan, Inner, Option(joinExprs.expr))(score.named) + } + + private def doFlatten(schema: StructType, separator: Char, prefixParts: Seq[String] = Seq.empty) + : Seq[Column] = { + schema.fields.flatMap { f => + val colNameParts = prefixParts :+ f.name + f.dataType match { + case st: StructType => + doFlatten(st, separator, colNameParts) + case _ => + col(colNameParts.mkString(".")).as(colNameParts.mkString(separator.toString)) :: Nil + } + } + } + + // Converts string representation of a character to actual character + @throws[IllegalArgumentException] + private def toChar(str: String): Char = { + if (str.length == 1) { + str.charAt(0) match { + case '$' | '_' | '.' => str.charAt(0) + case _ => throw new IllegalArgumentException( + "Must use '$', '_', or '.' for separator, but got " + str) + } + } else { + throw new IllegalArgumentException( + s"Separator cannot be more than one character: $str") + } + } + + /** + * Flattens a nested schema into a flat one. + * @group misc + * + * For example: + * {{{ + * scala> val df = Seq((0, (1, (3.0, "a")), (5, 0.9))).toDF() + * scala> df.printSchema + * root + * |-- _1: integer (nullable = false) + * |-- _2: struct (nullable = true) + * | |-- _1: integer (nullable = false) + * | |-- _2: struct (nullable = true) + * | | |-- _1: double (nullable = false) + * | | |-- _2: string (nullable = true) + * |-- _3: struct (nullable = true) + * | |-- _1: integer (nullable = false) + * | |-- _2: double (nullable = false) + * + * scala> df.flatten(separator = "$").printSchema + * root + * |-- _1: integer (nullable = false) + * |-- _2$_1: integer (nullable = true) + * |-- _2$_2$_1: double (nullable = true) + * |-- _2$_2$_2: string (nullable = true) + * |-- _3$_1: integer (nullable = true) + * |-- _3$_2: double (nullable = true) + * }}} + */ + def flatten(separator: String = "$"): DataFrame = + df.select(doFlatten(df.schema, toChar(separator)): _*) + + /** + * @see [[hivemall.dataset.LogisticRegressionDataGeneratorUDTF]] + * @group misc + */ + @scala.annotation.varargs + def lr_datagen(exprs: Column*): Dataset[Row] = withTypedPlan { + planHiveGenericUDTF( + df, + "hivemall.dataset.LogisticRegressionDataGeneratorUDTFWrapper", + "lr_datagen", + setMixServs(toHivemallFeatures(exprs)), + Seq("label", "features") + ) + } + + /** + * Returns all the columns as Seq[Column] in this [[DataFrame]]. + */ + private[sql] def cols: Seq[Column] = { + df.schema.fields.map(col => df.col(col.name)).toSeq + } + + /** + * :: Experimental :: + * If a parameter '-mix' does not exist in a 3rd argument, + * set it from an environmental variable + * 'HIVEMALL_MIX_SERVERS'. + * + * TODO: This could work if '--deploy-mode' has 'client'; + * otherwise, we need to set HIVEMALL_MIX_SERVERS + * in all possible spark workers. + */ + @Experimental + private[this] def setMixServs(exprs: Seq[Column]): Seq[Column] = { + val mixes = System.getenv("HIVEMALL_MIX_SERVERS") + if (mixes != null && !mixes.isEmpty()) { + val groupId = df.sqlContext.sparkContext.applicationId + "-" + UUID.randomUUID + logInfo(s"set '${mixes}' as default mix servers (session: ${groupId})") + exprs.size match { + case 2 => exprs :+ Column( + Literal.create(s"-mix ${mixes} -mix_session ${groupId}", StringType)) + /** TODO: Add codes in the case where exprs.size == 3. */ + case _ => exprs + } + } else { + exprs + } + } + + /** + * If the input is a [[Vector]], transform it into Hivemall features. + */ + @inline private[this] def toHivemallFeatures(exprs: Seq[Column]): Seq[Column] = { + df.select(exprs: _*).queryExecution.analyzed.schema.zip(exprs).map { + case (StructField(_, _: VectorUDT, _, _), c) => HivemallUtils.to_hivemall_features(c) + case (_, c) => c + } + } + + /** + * A convenient function to wrap a logical plan and produce a DataFrame. + */ + @inline private[this] def withTypedPlan(logicalPlan: => LogicalPlan): DataFrame = { + val queryExecution = _sparkSession.sessionState.executePlan(logicalPlan) + val outputSchema = queryExecution.sparkPlan.schema + new Dataset[Row](df.sparkSession, queryExecution, RowEncoder(outputSchema)) + } + + @inline private[this] def withTypedPlanInCustomStrategy(logicalPlan: => LogicalPlan) + : DataFrame = { + // Inject custom strategies + if (!_sparkSession.experimental.extraStrategies.contains(_strategy)) { + _sparkSession.experimental.extraStrategies = Seq(_strategy) + } + withTypedPlan(logicalPlan) + } +} + +object HivemallOps { + import internal.HivemallOpsImpl._ + + /** + * Implicitly inject the [[HivemallOps]] into [[DataFrame]]. + */ + implicit def dataFrameToHivemallOps(df: DataFrame): HivemallOps = + new HivemallOps(df) + + /** + * @see [[hivemall.HivemallVersionUDF]] + * @group misc + */ + def hivemall_version(): Column = withExpr { + planHiveUDF( + "hivemall.HivemallVersionUDF", + "hivemall_version", + Nil + ) + } + + /** + * @see [[hivemall.anomaly.ChangeFinderUDF]] + * @group anomaly + */ + @scala.annotation.varargs + def changefinder(exprs: Column*): Column = withExpr { + planHiveGenericUDF( + "hivemall.anomaly.ChangeFinderUDF", + "changefinder", + exprs + ) + } + + /** + * @see [[hivemall.anomaly.SingularSpectrumTransformUDF]] + * @group anomaly + */ + @scala.annotation.varargs + def sst(exprs: Column*): Column = withExpr { + planHiveGenericUDF( + "hivemall.anomaly.SingularSpectrumTransformUDF", + "sst", + exprs + ) + } + + /** + * @see [[hivemall.knn.similarity.CosineSimilarityUDF]] + * @group knn.similarity + */ + @scala.annotation.varargs + def cosine_sim(exprs: Column*): Column = withExpr { + planHiveGenericUDF( + "hivemall.knn.similarity.CosineSimilarityUDF", + "cosine_sim", + exprs + ) + } + + /** + * @see [[hivemall.knn.similarity.JaccardIndexUDF]] + * @group knn.similarity + */ + @scala.annotation.varargs + def jaccard(exprs: Column*): Column = withExpr { + planHiveUDF( + "hivemall.knn.similarity.JaccardIndexUDF", + "jaccard", + exprs + ) + } + + /** + * @see [[hivemall.knn.similarity.AngularSimilarityUDF]] + * @group knn.similarity + */ + @scala.annotation.varargs + def angular_similarity(exprs: Column*): Column = withExpr { + planHiveGenericUDF( + "hivemall.knn.similarity.AngularSimilarityUDF", + "angular_similarity", + exprs + ) + } + + /** + * @see [[hivemall.knn.similarity.EuclidSimilarity]] + * @group knn.similarity + */ + @scala.annotation.varargs + def euclid_similarity(exprs: Column*): Column = withExpr { + planHiveGenericUDF( + "hivemall.knn.similarity.EuclidSimilarity", + "euclid_similarity", + exprs + ) + } + + /** + * @see [[hivemall.knn.similarity.Distance2SimilarityUDF]] + * @group knn.similarity + */ + @scala.annotation.varargs + def distance2similarity(exprs: Column*): Column = withExpr { + // TODO: Need a wrapper class because of using unsupported types + planHiveGenericUDF( + "hivemall.knn.similarity.Distance2SimilarityUDF", + "distance2similarity", + exprs + ) + } + + /** + * @see [[hivemall.knn.distance.HammingDistanceUDF]] + * @group knn.distance + */ + @scala.annotation.varargs + def hamming_distance(exprs: Column*): Column = withExpr { + planHiveUDF( + "hivemall.knn.distance.HammingDistanceUDF", + "hamming_distance", + exprs + ) + } + + /** + * @see [[hivemall.knn.distance.PopcountUDF]] + * @group knn.distance + */ + @scala.annotation.varargs + def popcnt(exprs: Column*): Column = withExpr { + planHiveUDF( + "hivemall.knn.distance.PopcountUDF", + "popcnt", + exprs + ) + } + + /** + * @see [[hivemall.knn.distance.KLDivergenceUDF]] + * @group knn.distance + */ + @scala.annotation.varargs + def kld(exprs: Column*): Column = withExpr { + planHiveUDF( + "hivemall.knn.distance.KLDivergenceUDF", + "kld", + exprs + ) + } + + /** + * @see [[hivemall.knn.distance.EuclidDistanceUDF]] + * @group knn.distance + */ + @scala.annotation.varargs + def euclid_distance(exprs: Column*): Column = withExpr { + planHiveGenericUDF( + "hivemall.knn.distance.EuclidDistanceUDF", + "euclid_distance", + exprs + ) + } + + /** + * @see [[hivemall.knn.distance.CosineDistanceUDF]] + * @group knn.distance + */ + @scala.annotation.varargs + def cosine_distance(exprs: Column*): Column = withExpr { + planHiveGenericUDF( + "hivemall.knn.distance.CosineDistanceUDF", + "cosine_distance", + exprs + ) + } + + /** + * @see [[hivemall.knn.distance.AngularDistanceUDF]] + * @group knn.distance + */ + @scala.annotation.varargs + def angular_distance(exprs: Column*): Column = withExpr { + planHiveGenericUDF( + "hivemall.knn.distance.AngularDistanceUDF", + "angular_distance", + exprs + ) + } + + /** + * @see [[hivemall.knn.distance.ManhattanDistanceUDF]] + * @group knn.distance + */ + @scala.annotation.varargs + def manhattan_distance(exprs: Column*): Column = withExpr { + planHiveGenericUDF( + "hivemall.knn.distance.ManhattanDistanceUDF", + "manhattan_distance", + exprs + ) + } + + /** + * @see [[hivemall.knn.distance.MinkowskiDistanceUDF]] + * @group knn.distance + */ + @scala.annotation.varargs + def minkowski_distance (exprs: Column*): Column = withExpr { + planHiveGenericUDF( + "hivemall.knn.distance.MinkowskiDistanceUDF", + "minkowski_distance", + exprs + ) + } + + /** + * @see [[hivemall.knn.lsh.bBitMinHashUDF]] + * @group knn.lsh + */ + @scala.annotation.varargs + def bbit_minhash(exprs: Column*): Column = withExpr { + planHiveUDF( + "hivemall.knn.lsh.bBitMinHashUDF", + "bbit_minhash", + exprs + ) + } + + /** + * @see [[hivemall.knn.lsh.MinHashesUDFWrapper]] + * @group knn.lsh + */ + @scala.annotation.varargs + def minhashes(exprs: Column*): Column = withExpr { + planHiveGenericUDF( + "hivemall.knn.lsh.MinHashesUDFWrapper", + "minhashes", + exprs + ) + } + + /** + * Returns new features with `1.0` (bias) appended to the input features. + * @see [[hivemall.ftvec.AddBiasUDFWrapper]] + * @group ftvec + */ + def add_bias(expr: Column): Column = withExpr { + planHiveGenericUDF( + "hivemall.ftvec.AddBiasUDFWrapper", + "add_bias", + expr :: Nil + ) + } + + /** + * @see [[hivemall.ftvec.ExtractFeatureUDFWrapper]] + * @group ftvec + * + * TODO: This throws java.lang.ClassCastException because + * HiveInspectors.toInspector has a bug in spark. + * Need to fix it later. + */ + def extract_feature(expr: Column): Column = withExpr { + planHiveGenericUDF( + "hivemall.ftvec.ExtractFeatureUDFWrapper", + "extract_feature", + expr :: Nil + ) + }.as("feature") + + /** + * @see [[hivemall.ftvec.ExtractWeightUDFWrapper]] + * @group ftvec + * + * TODO: This throws java.lang.ClassCastException because + * HiveInspectors.toInspector has a bug in spark. + * Need to fix it later. + */ + def extract_weight(expr: Column): Column = withExpr { + planHiveGenericUDF( + "hivemall.ftvec.ExtractWeightUDFWrapper", + "extract_weight", + expr :: Nil + ) + }.as("value") + + /** + * @see [[hivemall.ftvec.AddFeatureIndexUDFWrapper]] + * @group ftvec + */ + def add_feature_index(expr: Column): Column = withExpr { + planHiveGenericUDF( + "hivemall.ftvec.AddFeatureIndexUDFWrapper", + "add_feature_index", + expr :: Nil + ) + } + + /** + * @see [[hivemall.ftvec.SortByFeatureUDFWrapper]] + * @group ftvec + */ + def sort_by_feature(expr: Column): Column = withExpr { + planHiveGenericUDF( + "hivemall.ftvec.SortByFeatureUDFWrapper", + "sort_by_feature", + expr :: Nil + ) + } + + /** + * @see [[hivemall.ftvec.hashing.MurmurHash3UDF]] + * @group ftvec.hashing + */ + def mhash(expr: Column): Column = withExpr { + planHiveUDF( + "hivemall.ftvec.hashing.MurmurHash3UDF", + "mhash", + expr :: Nil + ) + } + + /** + * @see [[hivemall.ftvec.hashing.Sha1UDF]] + * @group ftvec.hashing + */ + def sha1(expr: Column): Column = withExpr { + planHiveUDF( + "hivemall.ftvec.hashing.Sha1UDF", + "sha1", + expr :: Nil + ) + } + + /** + * @see [[hivemall.ftvec.hashing.ArrayHashValuesUDF]] + * @group ftvec.hashing + */ + @scala.annotation.varargs + def array_hash_values(exprs: Column*): Column = withExpr { + // TODO: Need a wrapper class because of using unsupported types + planHiveUDF( + "hivemall.ftvec.hashing.ArrayHashValuesUDF", + "array_hash_values", + exprs + ) + } + + /** + * @see [[hivemall.ftvec.hashing.ArrayPrefixedHashValuesUDF]] + * @group ftvec.hashing + */ + @scala.annotation.varargs + def prefixed_hash_values(exprs: Column*): Column = withExpr { + // TODO: Need a wrapper class because of using unsupported types + planHiveUDF( + "hivemall.ftvec.hashing.ArrayPrefixedHashValuesUDF", + "prefixed_hash_values", + exprs + ) + } + + /** + * @see [[hivemall.ftvec.scaling.RescaleUDF]] + * @group ftvec.scaling + */ + def rescale(value: Column, max: Column, min: Column): Column = withExpr { + planHiveUDF( + "hivemall.ftvec.scaling.RescaleUDF", + "rescale", + value.cast(FloatType) :: max :: min :: Nil + ) + } + + /** + * @see [[hivemall.ftvec.scaling.ZScoreUDF]] + * @group ftvec.scaling + */ + @scala.annotation.varargs + def zscore(exprs: Column*): Column = withExpr { + planHiveUDF( + "hivemall.ftvec.scaling.ZScoreUDF", + "zscore", + exprs + ) + } + + /** + * @see [[hivemall.ftvec.scaling.L2NormalizationUDFWrapper]] + * @group ftvec.scaling + */ + def normalize(expr: Column): Column = withExpr { + planHiveGenericUDF( + "hivemall.ftvec.scaling.L2NormalizationUDFWrapper", + "normalize", + expr :: Nil + ) + } + + /** + * @see [[hivemall.ftvec.selection.ChiSquareUDF]] + * @group ftvec.selection + */ + def chi2(observed: Column, expected: Column): Column = withExpr { + planHiveGenericUDF( + "hivemall.ftvec.selection.ChiSquareUDF", + "chi2", + Seq(observed, expected) + ) + } + + /** + * @see [[hivemall.ftvec.conv.ToDenseFeaturesUDF]] + * @group ftvec.conv + */ + @scala.annotation.varargs + def to_dense_features(exprs: Column*): Column = withExpr { + // TODO: Need a wrapper class because of using unsupported types + planHiveGenericUDF( + "hivemall.ftvec.conv.ToDenseFeaturesUDF", + "to_dense_features", + exprs + ) + } + + /** + * @see [[hivemall.ftvec.conv.ToSparseFeaturesUDF]] + * @group ftvec.conv + */ + @scala.annotation.varargs + def to_sparse_features(exprs: Column*): Column = withExpr { + // TODO: Need a wrapper class because of using unsupported types + planHiveGenericUDF( + "hivemall.ftvec.conv.ToSparseFeaturesUDF", + "to_sparse_features", + exprs + ) + } + + /** + * @see [[hivemall.ftvec.trans.VectorizeFeaturesUDF]] + * @group ftvec.trans + */ + @scala.annotation.varargs + def vectorize_features(exprs: Column*): Column = withExpr { + planHiveGenericUDF( + "hivemall.ftvec.trans.VectorizeFeaturesUDF", + "vectorize_features", + exprs + ) + } + + /** + * @see [[hivemall.ftvec.trans.CategoricalFeaturesUDF]] + * @group ftvec.trans + */ + @scala.annotation.varargs + def categorical_features(exprs: Column*): Column = withExpr { + planHiveGenericUDF( + "hivemall.ftvec.trans.CategoricalFeaturesUDF", + "categorical_features", + exprs + ) + } + + /** + * @see [[hivemall.ftvec.trans.IndexedFeatures]] + * @group ftvec.trans + */ + @scala.annotation.varargs + def indexed_features(exprs: Column*): Column = withExpr { + planHiveGenericUDF( + "hivemall.ftvec.trans.IndexedFeatures", + "indexed_features", + exprs + ) + } + + /** + * @see [[hivemall.ftvec.trans.QuantitativeFeaturesUDF]] + * @group ftvec.trans + */ + @scala.annotation.varargs + def quantitative_features(exprs: Column*): Column = withExpr { + planHiveGenericUDF( + "hivemall.ftvec.trans.QuantitativeFeaturesUDF", + "quantitative_features", + exprs + ) + } + + /** + * @see [[hivemall.smile.tools.TreePredictUDF]] + * @group misc + */ + @scala.annotation.varargs + def tree_predict(exprs: Column*): Column = withExpr { + planHiveGenericUDF( + "hivemall.smile.tools.TreePredictUDF", + "tree_predict", + exprs + ) + } + + /** + * @see [[hivemall.tools.array.SelectKBestUDF]] + * @group tools.array + */ + def select_k_best(X: Column, importanceList: Column, k: Column): Column = withExpr { + planHiveGenericUDF( + "hivemall.tools.array.SelectKBestUDF", + "select_k_best", + Seq(X, importanceList, k) + ) + } + + /** + * @see [[hivemall.tools.math.SigmoidGenericUDF]] + * @group misc + */ + def sigmoid(expr: Column): Column = { + val one: () => Literal = () => Literal.create(1.0, DoubleType) + Column(one()) / (Column(one()) + exp(-expr)) + } + + /** + * @see [[hivemall.tools.mapred.RowIdUDFWrapper]] + * @group misc + */ + def rowid(): Column = withExpr { + planHiveGenericUDF( + "hivemall.tools.mapred.RowIdUDFWrapper", + "rowid", + Nil + ) + }.as("rowid") + + /** + * Parses a column containing a CSV string into a [[StructType]] with the specified schema. + * Returns `null`, in the case of an unparseable string. + * @group misc + * + * @param e a string column containing CSV data. + * @param schema the schema to use when parsing the csv string + * @param options options to control how the csv is parsed. accepts the same options and the + * csv data source. + */ + def from_csv(e: Column, schema: StructType, options: Map[String, String]): Column = withExpr { + CsvToStruct(schema, options, e.expr) + } + + /** + * Parses a column containing a CSV string into a [[StructType]] with the specified schema. + * Returns `null`, in the case of an unparseable string. + * @group misc + * + * @param e a string column containing CSV data. + * @param schema the schema to use when parsing the json string + */ + def from_csv(e: Column, schema: StructType): Column = + from_csv(e, schema, Map.empty[String, String]) + + /** + * Converts a column containing a [[StructType]] into a CSV string with the specified schema. + * Throws an exception, in the case of an unsupported type. + * @group misc + * + * @param e a struct column. + * @param options options to control how the struct column is converted into a json string. + * accepts the same options and the json data source. + */ + def to_csv(e: Column, options: Map[String, String]): Column = withExpr { + StructToCsv(options, e.expr) + } + + /** + * Converts a column containing a [[StructType]] into a CSV string with the specified schema. + * Throws an exception, in the case of an unsupported type. + * @group misc + * + * @param e a struct column. + */ + def to_csv(e: Column): Column = to_csv(e, Map.empty[String, String]) + + /** + * A convenient function to wrap an expression and produce a Column. + */ + @inline private def withExpr(expr: Expression): Column = Column(expr) +} diff --git a/spark/spark-2.2/src/main/scala/org/apache/spark/sql/hive/HivemallUtils.scala b/spark/spark-2.2/src/main/scala/org/apache/spark/sql/hive/HivemallUtils.scala new file mode 100644 index 000000000..70cf00b92 --- /dev/null +++ b/spark/spark-2.2/src/main/scala/org/apache/spark/sql/hive/HivemallUtils.scala @@ -0,0 +1,146 @@ +/* + * 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 org.apache.spark.ml.linalg.{BLAS, DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.expressions.UserDefinedFunction +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ + +object HivemallUtils { + + // # of maximum dimensions for feature vectors + private[this] val maxDims = 100000000 + + /** + * Check whether the given schema contains a column of the required data type. + * @param colName column name + * @param dataType required column data type + */ + private[this] def checkColumnType(schema: StructType, colName: String, dataType: DataType) + : Unit = { + val actualDataType = schema(colName).dataType + require(actualDataType.equals(dataType), + s"Column $colName must be of type $dataType but was actually $actualDataType.") + } + + def to_vector_func(dense: Boolean, dims: Int): Seq[String] => Vector = { + if (dense) { + // Dense features + i: Seq[String] => { + val features = new Array[Double](dims) + i.map { ft => + val s = ft.split(":").ensuring(_.size == 2) + features(s(0).toInt) = s(1).toDouble + } + Vectors.dense(features) + } + } else { + // Sparse features + i: Seq[String] => { + val features = i.map { ft => + // val s = ft.split(":").ensuring(_.size == 2) + val s = ft.split(":") + (s(0).toInt, s(1).toDouble) + } + Vectors.sparse(dims, features) + } + } + } + + def to_hivemall_features_func(): Vector => Array[String] = { + case dv: DenseVector => + dv.values.zipWithIndex.map { + case (value, index) => s"$index:$value" + } + case sv: SparseVector => + sv.values.zip(sv.indices).map { + case (value, index) => s"$index:$value" + } + case v => + throw new IllegalArgumentException(s"Do not support vector type ${v.getClass}") + } + + def append_bias_func(): Vector => Vector = { + case dv: DenseVector => + val inputValues = dv.values + val inputLength = inputValues.length + val outputValues = Array.ofDim[Double](inputLength + 1) + System.arraycopy(inputValues, 0, outputValues, 0, inputLength) + outputValues(inputLength) = 1.0 + Vectors.dense(outputValues) + case sv: SparseVector => + val inputValues = sv.values + val inputIndices = sv.indices + val inputValuesLength = inputValues.length + val dim = sv.size + val outputValues = Array.ofDim[Double](inputValuesLength + 1) + val outputIndices = Array.ofDim[Int](inputValuesLength + 1) + System.arraycopy(inputValues, 0, outputValues, 0, inputValuesLength) + System.arraycopy(inputIndices, 0, outputIndices, 0, inputValuesLength) + outputValues(inputValuesLength) = 1.0 + outputIndices(inputValuesLength) = dim + Vectors.sparse(dim + 1, outputIndices, outputValues) + case v => + throw new IllegalArgumentException(s"Do not support vector type ${v.getClass}") + } + + /** + * Transforms Hivemall features into a [[Vector]]. + */ + def to_vector(dense: Boolean = false, dims: Int = maxDims): UserDefinedFunction = { + udf(to_vector_func(dense, dims)) + } + + /** + * Transforms a [[Vector]] into Hivemall features. + */ + def to_hivemall_features: UserDefinedFunction = udf(to_hivemall_features_func) + + /** + * Returns a new [[Vector]] with `1.0` (bias) appended to the input [[Vector]]. + * @group ftvec + */ + def append_bias: UserDefinedFunction = udf(append_bias_func) + + /** + * Builds a [[Vector]]-based model from a table of Hivemall models + */ + def vectorized_model(df: DataFrame, dense: Boolean = false, dims: Int = maxDims) + : UserDefinedFunction = { + checkColumnType(df.schema, "feature", StringType) + checkColumnType(df.schema, "weight", DoubleType) + + import df.sqlContext.implicits._ + val intercept = df + .where($"feature" === "0") + .select($"weight") + .map { case Row(weight: Double) => weight} + .reduce(_ + _) + val weights = to_vector_func(dense, dims)( + df.select($"feature", $"weight") + .where($"feature" !== "0") + .map { case Row(label: String, feature: Double) => s"${label}:$feature"} + .collect.toSeq) + + udf((input: Vector) => BLAS.dot(input, weights) + intercept) + } +} diff --git a/spark/spark-2.2/src/main/scala/org/apache/spark/sql/hive/internal/HivemallOpsImpl.scala b/spark/spark-2.2/src/main/scala/org/apache/spark/sql/hive/internal/HivemallOpsImpl.scala new file mode 100644 index 000000000..179b146d8 --- /dev/null +++ b/spark/spark-2.2/src/main/scala/org/apache/spark/sql/hive/internal/HivemallOpsImpl.scala @@ -0,0 +1,79 @@ +/* + * 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.internal + +import org.apache.spark.internal.Logging +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.logical.{Generate, LogicalPlan} +import org.apache.spark.sql.hive._ +import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper + +/** + * This is an implementation class for [[org.apache.spark.sql.hive.HivemallOps]]. + * This class mainly uses the internal Spark classes (e.g., `Generate` and `HiveGenericUDTF`) that + * have unstable interfaces (so, these interfaces may evolve in upcoming releases). + * Therefore, the objective of this class is to extract these unstable parts + * from [[org.apache.spark.sql.hive.HivemallOps]]. + */ +private[hive] object HivemallOpsImpl extends Logging { + + def planHiveUDF( + className: String, + funcName: String, + argumentExprs: Seq[Column]): Expression = { + HiveSimpleUDF( + name = funcName, + funcWrapper = new HiveFunctionWrapper(className), + children = argumentExprs.map(_.expr) + ) + } + + def planHiveGenericUDF( + className: String, + funcName: String, + argumentExprs: Seq[Column]): Expression = { + HiveGenericUDF( + name = funcName, + funcWrapper = new HiveFunctionWrapper(className), + children = argumentExprs.map(_.expr) + ) + } + + def planHiveGenericUDTF( + df: DataFrame, + className: String, + funcName: String, + argumentExprs: Seq[Column], + outputAttrNames: Seq[String]): LogicalPlan = { + Generate( + generator = HiveGenericUDTF( + name = funcName, + funcWrapper = new HiveFunctionWrapper(className), + children = argumentExprs.map(_.expr) + ), + join = false, + outer = false, + qualifier = None, + generatorOutput = outputAttrNames.map(UnresolvedAttribute(_)), + child = df.logicalPlan) + } +} diff --git a/spark/spark-2.2/src/main/scala/org/apache/spark/sql/hive/source/XGBoostFileFormat.scala b/spark/spark-2.2/src/main/scala/org/apache/spark/sql/hive/source/XGBoostFileFormat.scala new file mode 100644 index 000000000..65cdf2448 --- /dev/null +++ b/spark/spark-2.2/src/main/scala/org/apache/spark/sql/hive/source/XGBoostFileFormat.scala @@ -0,0 +1,163 @@ +/* + * 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.source + +import java.io.File +import java.io.IOException +import java.net.URI + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, FSDataInputStream, Path} +import org.apache.hadoop.io.IOUtils +import org.apache.hadoop.io.compress.GzipCodec +import org.apache.hadoop.mapreduce._ +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat +import org.apache.hadoop.util.ReflectionUtils + +import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.encoders.RowEncoder +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types._ +import org.apache.spark.util.SerializableConfiguration + +private[source] final class XGBoostOutputWriter( + path: String, + dataSchema: StructType, + context: TaskAttemptContext) + extends OutputWriter { + + private val hadoopConf = new SerializableConfiguration(new Configuration()) + + override def write(row: InternalRow): Unit = { + val fields = row.toSeq(dataSchema) + val model = fields(1).asInstanceOf[Array[Byte]] + val filePath = new Path(new URI(s"$path")) + val fs = filePath.getFileSystem(hadoopConf.value) + val outputFile = fs.create(filePath) + outputFile.write(model) + outputFile.close() + } + + override def close(): Unit = {} +} + +object XGBoostOutputWriter { + + /** Returns the compression codec extension to be used in a file name, e.g. ".gzip"). */ + def getCompressionExtension(context: TaskAttemptContext): String = { + if (FileOutputFormat.getCompressOutput(context)) { + val codecClass = FileOutputFormat.getOutputCompressorClass(context, classOf[GzipCodec]) + ReflectionUtils.newInstance(codecClass, context.getConfiguration).getDefaultExtension + } else { + "" + } + } +} + +final class XGBoostFileFormat extends FileFormat with DataSourceRegister { + + override def shortName(): String = "libxgboost" + + override def toString: String = "XGBoost" + + private def verifySchema(dataSchema: StructType): Unit = { + if ( + dataSchema.size != 2 || + !dataSchema(0).dataType.sameType(StringType) || + !dataSchema(1).dataType.sameType(BinaryType) + ) { + throw new IOException(s"Illegal schema for XGBoost data, schema=$dataSchema") + } + } + + override def inferSchema( + sparkSession: SparkSession, + options: Map[String, String], + files: Seq[FileStatus]): Option[StructType] = { + Some( + StructType( + StructField("model_id", StringType, nullable = false) :: + StructField("pred_model", BinaryType, nullable = false) :: Nil) + ) + } + + override def prepareWrite( + sparkSession: SparkSession, + job: Job, + options: Map[String, String], + dataSchema: StructType): OutputWriterFactory = { + new OutputWriterFactory { + override def newInstance( + path: String, + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = { + new XGBoostOutputWriter(path, dataSchema, context) + } + + override def getFileExtension(context: TaskAttemptContext): String = { + XGBoostOutputWriter.getCompressionExtension(context) + ".xgboost" + } + } + } + + override def buildReader( + sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { + verifySchema(dataSchema) + val broadcastedHadoopConf = + sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) + + (file: PartitionedFile) => { + val model = new Array[Byte](file.length.asInstanceOf[Int]) + val filePath = new Path(new URI(file.filePath)) + val fs = filePath.getFileSystem(broadcastedHadoopConf.value.value) + + var in: FSDataInputStream = null + try { + in = fs.open(filePath) + IOUtils.readFully(in, model, 0, model.length) + } finally { + IOUtils.closeStream(in) + } + + val converter = RowEncoder(dataSchema) + val fullOutput = dataSchema.map { f => + AttributeReference(f.name, f.dataType, f.nullable, f.metadata)() + } + val requiredOutput = fullOutput.filter { a => + requiredSchema.fieldNames.contains(a.name) + } + val requiredColumns = GenerateUnsafeProjection.generate(requiredOutput, fullOutput) + (requiredColumns( + converter.toRow(Row(new File(file.filePath).getName, model))) + :: Nil + ).toIterator + } + } +} diff --git a/spark/spark-2.2/src/test/resources/data/files/README.md b/spark/spark-2.2/src/test/resources/data/files/README.md new file mode 100644 index 000000000..0fd029918 --- /dev/null +++ b/spark/spark-2.2/src/test/resources/data/files/README.md @@ -0,0 +1,3 @@ +The files in this dir exist for preventing exceptions in o.a.s.sql.hive.test.TESTHive. +We need to fix this issue in future. + diff --git a/spark/spark-2.2/src/test/resources/data/files/complex.seq b/spark/spark-2.2/src/test/resources/data/files/complex.seq new file mode 100644 index 000000000..e69de29bb diff --git a/spark/spark-2.2/src/test/resources/data/files/episodes.avro b/spark/spark-2.2/src/test/resources/data/files/episodes.avro new file mode 100644 index 000000000..e69de29bb diff --git a/spark/spark-2.2/src/test/resources/data/files/json.txt b/spark/spark-2.2/src/test/resources/data/files/json.txt new file mode 100644 index 000000000..e69de29bb diff --git a/spark/spark-2.2/src/test/resources/data/files/kv1.txt b/spark/spark-2.2/src/test/resources/data/files/kv1.txt new file mode 100644 index 000000000..e69de29bb diff --git a/spark/spark-2.2/src/test/resources/data/files/kv3.txt b/spark/spark-2.2/src/test/resources/data/files/kv3.txt new file mode 100644 index 000000000..e69de29bb diff --git a/spark/spark-2.2/src/test/resources/log4j.properties b/spark/spark-2.2/src/test/resources/log4j.properties new file mode 100644 index 000000000..1db11f092 --- /dev/null +++ b/spark/spark-2.2/src/test/resources/log4j.properties @@ -0,0 +1,7 @@ +# Set everything to be logged to the console +log4j.rootCategory=FATAL, console +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n + diff --git a/spark/spark-2.2/src/test/scala/hivemall/mix/server/MixServerSuite.scala b/spark/spark-2.2/src/test/scala/hivemall/mix/server/MixServerSuite.scala new file mode 100644 index 000000000..9bbd3f0f4 --- /dev/null +++ b/spark/spark-2.2/src/test/scala/hivemall/mix/server/MixServerSuite.scala @@ -0,0 +1,124 @@ +/* + * 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 hivemall.mix.server + +import java.util.Random +import java.util.concurrent.{Executors, ExecutorService, TimeUnit} +import java.util.logging.Logger + +import hivemall.mix.MixMessage.MixEventName +import hivemall.mix.client.MixClient +import hivemall.mix.server.MixServer.ServerState +import hivemall.model.{DenseModel, PredictionModel, WeightValue} +import hivemall.utils.io.IOUtils +import hivemall.utils.lang.CommandLineUtils +import hivemall.utils.net.NetUtils +import org.scalatest.{BeforeAndAfter, FunSuite} + +class MixServerSuite extends FunSuite with BeforeAndAfter { + + private[this] var server: MixServer = _ + private[this] var executor : ExecutorService = _ + private[this] var port: Int = _ + + private[this] val rand = new Random(43) + private[this] val counter = Stream.from(0).iterator + + private[this] val eachTestTime = 100 + private[this] val logger = + Logger.getLogger(classOf[MixServerSuite].getName) + + before { + this.port = NetUtils.getAvailablePort + this.server = new MixServer( + CommandLineUtils.parseOptions( + Array("-port", s"${port}", "-sync_threshold", "3"), + MixServer.getOptions() + ) + ) + this.executor = Executors.newSingleThreadExecutor + this.executor.submit(server) + var retry = 0 + while (server.getState() != ServerState.RUNNING && retry < 50) { + Thread.sleep(1000L) + retry += 1 + } + assert(server.getState == ServerState.RUNNING) + } + + after { this.executor.shutdown() } + + private[this] def clientDriver( + groupId: String, model: PredictionModel, numMsg: Int = 1000000): Unit = { + var client: MixClient = null + try { + client = new MixClient(MixEventName.average, groupId, s"localhost:${port}", false, 2, model) + model.configureMix(client, false) + model.configureClock() + + for (_ <- 0 until numMsg) { + val feature = Integer.valueOf(rand.nextInt(model.size)) + model.set(feature, new WeightValue(1.0f)) + } + + while (true) { Thread.sleep(eachTestTime * 1000 + 100L) } + assert(model.getNumMixed > 0) + } finally { + IOUtils.closeQuietly(client) + } + } + + private[this] def fixedGroup: (String, () => String) = + ("fixed", () => "fixed") + private[this] def uniqueGroup: (String, () => String) = + ("unique", () => s"${counter.next}") + + Seq(65536).map { ndims => + Seq(4).map { nclient => + Seq(fixedGroup, uniqueGroup).map { id => + val testName = s"dense-dim:${ndims}-clinet:${nclient}-${id._1}" + ignore(testName) { + val clients = Executors.newCachedThreadPool() + val numClients = nclient + val models = (0 until numClients).map(i => new DenseModel(ndims, false)) + (0 until numClients).map { i => + clients.submit(new Runnable() { + override def run(): Unit = { + try { + clientDriver( + s"${testName}-${id._2}", + models(i) + ) + } catch { + case e: InterruptedException => + assert(false, e.getMessage) + } + } + }) + } + clients.awaitTermination(eachTestTime, TimeUnit.SECONDS) + clients.shutdown() + val nMixes = models.map(d => d.getNumMixed).reduce(_ + _) + logger.info(s"${testName} --> ${(nMixes + 0.0) / eachTestTime} mixes/s") + } + } + } + } +} diff --git a/spark/spark-2.2/src/test/scala/hivemall/tools/RegressionDatagenSuite.scala b/spark/spark-2.2/src/test/scala/hivemall/tools/RegressionDatagenSuite.scala new file mode 100644 index 000000000..c12727610 --- /dev/null +++ b/spark/spark-2.2/src/test/scala/hivemall/tools/RegressionDatagenSuite.scala @@ -0,0 +1,33 @@ +/* + * 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 hivemall.tools + +import org.scalatest.FunSuite + +import org.apache.spark.sql.hive.test.TestHive + +class RegressionDatagenSuite extends FunSuite { + + test("datagen") { + val df = RegressionDatagen.exec( + TestHive, min_examples = 10000, n_features = 100, n_dims = 65536, dense = false, cl = true) + assert(df.count() >= 10000) + } +} diff --git a/spark/spark-2.2/src/test/scala/org/apache/spark/SparkFunSuite.scala b/spark/spark-2.2/src/test/scala/org/apache/spark/SparkFunSuite.scala new file mode 100644 index 000000000..ed1bb6aaf --- /dev/null +++ b/spark/spark-2.2/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -0,0 +1,51 @@ +/* + * 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 + +// scalastyle:off +import org.scalatest.{FunSuite, Outcome} + +import org.apache.spark.internal.Logging + +/** + * Base abstract class for all unit tests in Spark for handling common functionality. + */ +private[spark] abstract class SparkFunSuite extends FunSuite with Logging { +// scalastyle:on + + /** + * Log the suite name and the test name before and after each test. + * + * Subclasses should never override this method. If they wish to run + * custom code before and after each test, they should mix in the + * {{org.scalatest.BeforeAndAfter}} trait instead. + */ + final protected override def withFixture(test: NoArgTest): Outcome = { + val testName = test.text + val suiteName = this.getClass.getName + val shortSuiteName = suiteName.replaceAll("org.apache.spark", "o.a.s") + try { + logInfo(s"\n\n===== TEST OUTPUT FOR $shortSuiteName: '$testName' =====\n") + test() + } finally { + logInfo(s"\n\n===== FINISHED $shortSuiteName: '$testName' =====\n") + } + } +} diff --git a/spark/spark-2.2/src/test/scala/org/apache/spark/ml/feature/HivemallLabeledPointSuite.scala b/spark/spark-2.2/src/test/scala/org/apache/spark/ml/feature/HivemallLabeledPointSuite.scala new file mode 100644 index 000000000..903dc0ae3 --- /dev/null +++ b/spark/spark-2.2/src/test/scala/org/apache/spark/ml/feature/HivemallLabeledPointSuite.scala @@ -0,0 +1,36 @@ +/* + * 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.ml.feature + +import org.apache.spark.SparkFunSuite + +class HivemallLabeledPointSuite extends SparkFunSuite { + + test("toString") { + val lp = HivemallLabeledPoint(1.0f, Seq("1:0.5", "3:0.3", "8:0.1")) + assert(lp.toString === "1.0,[1:0.5,3:0.3,8:0.1]") + } + + test("parse") { + val lp = HivemallLabeledPoint.parse("1.0,[1:0.5,3:0.3,8:0.1]") + assert(lp.label === 1.0) + assert(lp.features === Seq("1:0.5", "3:0.3", "8:0.1")) + } +} diff --git a/spark/spark-2.2/src/test/scala/org/apache/spark/sql/QueryTest.scala b/spark/spark-2.2/src/test/scala/org/apache/spark/sql/QueryTest.scala new file mode 100644 index 000000000..c9d0ba02d --- /dev/null +++ b/spark/spark-2.2/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -0,0 +1,360 @@ +/* + * 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 + +import java.util.{ArrayDeque, Locale, TimeZone} + +import scala.collection.JavaConverters._ +import scala.util.control.NonFatal + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate.ImperativeAggregate +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.trees.TreeNode +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression +import org.apache.spark.sql.execution.columnar.InMemoryRelation +import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.execution.streaming.MemoryPlan +import org.apache.spark.sql.types.{Metadata, ObjectType} + + +abstract class QueryTest extends PlanTest { + + protected def spark: SparkSession + + // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) + TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) + // Add Locale setting + Locale.setDefault(Locale.US) + + /** + * Runs the plan and makes sure the answer contains all of the keywords. + */ + def checkKeywordsExist(df: DataFrame, keywords: String*): Unit = { + val outputs = df.collect().map(_.mkString).mkString + for (key <- keywords) { + assert(outputs.contains(key), s"Failed for $df ($key doesn't exist in result)") + } + } + + /** + * Runs the plan and makes sure the answer does NOT contain any of the keywords. + */ + def checkKeywordsNotExist(df: DataFrame, keywords: String*): Unit = { + val outputs = df.collect().map(_.mkString).mkString + for (key <- keywords) { + assert(!outputs.contains(key), s"Failed for $df ($key existed in the result)") + } + } + + /** + * Evaluates a dataset to make sure that the result of calling collect matches the given + * expected answer. + */ + protected def checkDataset[T]( + ds: => Dataset[T], + expectedAnswer: T*): Unit = { + val result = getResult(ds) + + if (!compare(result.toSeq, expectedAnswer)) { + fail( + s""" + |Decoded objects do not match expected objects: + |expected: $expectedAnswer + |actual: ${result.toSeq} + |${ds.exprEnc.deserializer.treeString} + """.stripMargin) + } + } + + /** + * Evaluates a dataset to make sure that the result of calling collect matches the given + * expected answer, after sort. + */ + protected def checkDatasetUnorderly[T : Ordering]( + ds: => Dataset[T], + expectedAnswer: T*): Unit = { + val result = getResult(ds) + + if (!compare(result.toSeq.sorted, expectedAnswer.sorted)) { + fail( + s""" + |Decoded objects do not match expected objects: + |expected: $expectedAnswer + |actual: ${result.toSeq} + |${ds.exprEnc.deserializer.treeString} + """.stripMargin) + } + } + + private def getResult[T](ds: => Dataset[T]): Array[T] = { + val analyzedDS = try ds catch { + case ae: AnalysisException => + if (ae.plan.isDefined) { + fail( + s""" + |Failed to analyze query: $ae + |${ae.plan.get} + | + |${stackTraceToString(ae)} + """.stripMargin) + } else { + throw ae + } + } + assertEmptyMissingInput(analyzedDS) + + try ds.collect() catch { + case e: Exception => + fail( + s""" + |Exception collecting dataset as objects + |${ds.exprEnc} + |${ds.exprEnc.deserializer.treeString} + |${ds.queryExecution} + """.stripMargin, e) + } + } + + private def compare(obj1: Any, obj2: Any): Boolean = (obj1, obj2) match { + case (null, null) => true + case (null, _) => false + case (_, null) => false + case (a: Array[_], b: Array[_]) => + a.length == b.length && a.zip(b).forall { case (l, r) => compare(l, r)} + case (a: Iterable[_], b: Iterable[_]) => + a.size == b.size && a.zip(b).forall { case (l, r) => compare(l, r)} + case (a, b) => a == b + } + + /** + * Runs the plan and makes sure the answer matches the expected result. + * + * @param df the [[DataFrame]] to be executed + * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. + */ + protected def checkAnswer(df: => DataFrame, expectedAnswer: Seq[Row]): Unit = { + val analyzedDF = try df catch { + case ae: AnalysisException => + if (ae.plan.isDefined) { + fail( + s""" + |Failed to analyze query: $ae + |${ae.plan.get} + | + |${stackTraceToString(ae)} + |""".stripMargin) + } else { + throw ae + } + } + + assertEmptyMissingInput(analyzedDF) + + QueryTest.checkAnswer(analyzedDF, expectedAnswer) match { + case Some(errorMessage) => fail(errorMessage) + case None => + } + } + + protected def checkAnswer(df: => DataFrame, expectedAnswer: Row): Unit = { + checkAnswer(df, Seq(expectedAnswer)) + } + + protected def checkAnswer(df: => DataFrame, expectedAnswer: DataFrame): Unit = { + checkAnswer(df, expectedAnswer.collect()) + } + + /** + * Runs the plan and makes sure the answer is within absTol of the expected result. + * + * @param dataFrame the [[DataFrame]] to be executed + * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. + * @param absTol the absolute tolerance between actual and expected answers. + */ + protected def checkAggregatesWithTol(dataFrame: DataFrame, + expectedAnswer: Seq[Row], + absTol: Double): Unit = { + // TODO: catch exceptions in data frame execution + val actualAnswer = dataFrame.collect() + require(actualAnswer.length == expectedAnswer.length, + s"actual num rows ${actualAnswer.length} != expected num of rows ${expectedAnswer.length}") + + actualAnswer.zip(expectedAnswer).foreach { + case (actualRow, expectedRow) => + QueryTest.checkAggregatesWithTol(actualRow, expectedRow, absTol) + } + } + + protected def checkAggregatesWithTol(dataFrame: DataFrame, + expectedAnswer: Row, + absTol: Double): Unit = { + checkAggregatesWithTol(dataFrame, Seq(expectedAnswer), absTol) + } + + /** + * Asserts that a given [[Dataset]] will be executed using the given number of cached results. + */ + def assertCached(query: Dataset[_], numCachedTables: Int = 1): Unit = { + val planWithCaching = query.queryExecution.withCachedData + val cachedData = planWithCaching collect { + case cached: InMemoryRelation => cached + } + + assert( + cachedData.size == numCachedTables, + s"Expected query to contain $numCachedTables, but it actually had ${cachedData.size}\n" + + planWithCaching) + } + + /** + * Asserts that a given [[Dataset]] does not have missing inputs in all the analyzed plans. + */ + def assertEmptyMissingInput(query: Dataset[_]): Unit = { + assert(query.queryExecution.analyzed.missingInput.isEmpty, + s"The analyzed logical plan has missing inputs:\n${query.queryExecution.analyzed}") + assert(query.queryExecution.optimizedPlan.missingInput.isEmpty, + s"The optimized logical plan has missing inputs:\n${query.queryExecution.optimizedPlan}") + assert(query.queryExecution.executedPlan.missingInput.isEmpty, + s"The physical plan has missing inputs:\n${query.queryExecution.executedPlan}") + } +} + +object QueryTest { + /** + * Runs the plan and makes sure the answer matches the expected result. + * If there was exception during the execution or the contents of the DataFrame does not + * match the expected result, an error message will be returned. Otherwise, a [[None]] will + * be returned. + * + * @param df the [[DataFrame]] to be executed + * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. + * @param checkToRDD whether to verify deserialization to an RDD. This runs the query twice. + */ + def checkAnswer( + df: DataFrame, + expectedAnswer: Seq[Row], + checkToRDD: Boolean = true): Option[String] = { + val isSorted = df.logicalPlan.collect { case s: logical.Sort => s }.nonEmpty + if (checkToRDD) { + df.rdd.count() // Also attempt to deserialize as an RDD [SPARK-15791] + } + + val sparkAnswer = try df.collect().toSeq catch { + case e: Exception => + val errorMessage = + s""" + |Exception thrown while executing query: + |${df.queryExecution} + |== Exception == + |$e + |${org.apache.spark.sql.catalyst.util.stackTraceToString(e)} + """.stripMargin + return Some(errorMessage) + } + + sameRows(expectedAnswer, sparkAnswer, isSorted).map { results => + s""" + |Results do not match for query: + |Timezone: ${TimeZone.getDefault} + |Timezone Env: ${sys.env.getOrElse("TZ", "")} + | + |${df.queryExecution} + |== Results == + |$results + """.stripMargin + } + } + + + def prepareAnswer(answer: Seq[Row], isSorted: Boolean): Seq[Row] = { + // Converts data to types that we can do equality comparison using Scala collections. + // For BigDecimal type, the Scala type has a better definition of equality test (similar to + // Java's java.math.BigDecimal.compareTo). + // For binary arrays, we convert it to Seq to avoid of calling java.util.Arrays.equals for + // equality test. + val converted: Seq[Row] = answer.map(prepareRow) + if (!isSorted) converted.sortBy(_.toString()) else converted + } + + // We need to call prepareRow recursively to handle schemas with struct types. + def prepareRow(row: Row): Row = { + Row.fromSeq(row.toSeq.map { + case null => null + case d: java.math.BigDecimal => BigDecimal(d) + // Convert array to Seq for easy equality check. + case b: Array[_] => b.toSeq + case r: Row => prepareRow(r) + case o => o + }) + } + + def sameRows( + expectedAnswer: Seq[Row], + sparkAnswer: Seq[Row], + isSorted: Boolean = false): Option[String] = { + if (prepareAnswer(expectedAnswer, isSorted) != prepareAnswer(sparkAnswer, isSorted)) { + val errorMessage = + s""" + |== Results == + |${sideBySide( + s"== Correct Answer - ${expectedAnswer.size} ==" +: + prepareAnswer(expectedAnswer, isSorted).map(_.toString()), + s"== Spark Answer - ${sparkAnswer.size} ==" +: + prepareAnswer(sparkAnswer, isSorted).map(_.toString())).mkString("\n")} + """.stripMargin + return Some(errorMessage) + } + None + } + + /** + * Runs the plan and makes sure the answer is within absTol of the expected result. + * + * @param actualAnswer the actual result in a [[Row]]. + * @param expectedAnswer the expected result in a[[Row]]. + * @param absTol the absolute tolerance between actual and expected answers. + */ + protected def checkAggregatesWithTol(actualAnswer: Row, expectedAnswer: Row, absTol: Double) = { + require(actualAnswer.length == expectedAnswer.length, + s"actual answer length ${actualAnswer.length} != " + + s"expected answer length ${expectedAnswer.length}") + + // TODO: support other numeric types besides Double + // TODO: support struct types? + actualAnswer.toSeq.zip(expectedAnswer.toSeq).foreach { + case (actual: Double, expected: Double) => + assert(math.abs(actual - expected) < absTol, + s"actual answer $actual not within $absTol of correct answer $expected") + case (actual, expected) => + assert(actual == expected, s"$actual did not equal $expected") + } + } + + def checkAnswer(df: DataFrame, expectedAnswer: java.util.List[Row]): String = { + checkAnswer(df, expectedAnswer.asScala) match { + case Some(errorMessage) => errorMessage + case None => null + } + } +} diff --git a/spark/spark-2.2/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/spark/spark-2.2/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala new file mode 100644 index 000000000..f44428c35 --- /dev/null +++ b/spark/spark-2.2/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -0,0 +1,135 @@ +/* + * 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.catalyst.plans + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.internal.SQLConf + +/** + * Provides helper methods for comparing plans. + */ +abstract class PlanTest extends SparkFunSuite with PredicateHelper { + + protected val conf = new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true) + + /** + * Since attribute references are given globally unique ids during analysis, + * we must normalize them to check if two different queries are identical. + */ + protected def normalizeExprIds(plan: LogicalPlan) = { + plan transformAllExpressions { + case s: ScalarSubquery => + s.copy(exprId = ExprId(0)) + case e: Exists => + e.copy(exprId = ExprId(0)) + case l: ListQuery => + l.copy(exprId = ExprId(0)) + case a: AttributeReference => + AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(0)) + case a: Alias => + Alias(a.child, a.name)(exprId = ExprId(0)) + case ae: AggregateExpression => + ae.copy(resultId = ExprId(0)) + } + } + + /** + * Normalizes plans: + * - Filter the filter conditions that appear in a plan. For instance, + * ((expr 1 && expr 2) && expr 3), (expr 1 && expr 2 && expr 3), (expr 3 && (expr 1 && expr 2) + * etc., will all now be equivalent. + * - Sample the seed will replaced by 0L. + * - Join conditions will be resorted by hashCode. + */ + protected def normalizePlan(plan: LogicalPlan): LogicalPlan = { + plan transform { + case filter @ Filter(condition: Expression, child: LogicalPlan) => + Filter(splitConjunctivePredicates(condition).map(rewriteEqual(_)).sortBy(_.hashCode()) + .reduce(And), child) + case sample: Sample => + sample.copy(seed = 0L)(true) + case join @ Join(left, right, joinType, condition) if condition.isDefined => + val newCondition = + splitConjunctivePredicates(condition.get).map(rewriteEqual(_)).sortBy(_.hashCode()) + .reduce(And) + Join(left, right, joinType, Some(newCondition)) + } + } + + /** + * Rewrite [[EqualTo]] and [[EqualNullSafe]] operator to keep order. The following cases will be + * equivalent: + * 1. (a = b), (b = a); + * 2. (a <=> b), (b <=> a). + */ + private def rewriteEqual(condition: Expression): Expression = condition match { + case eq @ EqualTo(l: Expression, r: Expression) => + Seq(l, r).sortBy(_.hashCode()).reduce(EqualTo) + case eq @ EqualNullSafe(l: Expression, r: Expression) => + Seq(l, r).sortBy(_.hashCode()).reduce(EqualNullSafe) + case _ => condition // Don't reorder. + } + + /** Fails the test if the two plans do not match */ + protected def comparePlans(plan1: LogicalPlan, plan2: LogicalPlan) { + val normalized1 = normalizePlan(normalizeExprIds(plan1)) + val normalized2 = normalizePlan(normalizeExprIds(plan2)) + if (normalized1 != normalized2) { + fail( + s""" + |== FAIL: Plans do not match === + |${sideBySide(normalized1.treeString, normalized2.treeString).mkString("\n")} + """.stripMargin) + } + } + + /** Fails the test if the two expressions do not match */ + protected def compareExpressions(e1: Expression, e2: Expression): Unit = { + comparePlans(Filter(e1, OneRowRelation), Filter(e2, OneRowRelation)) + } + + /** Fails the test if the join order in the two plans do not match */ + protected def compareJoinOrder(plan1: LogicalPlan, plan2: LogicalPlan) { + val normalized1 = normalizePlan(normalizeExprIds(plan1)) + val normalized2 = normalizePlan(normalizeExprIds(plan2)) + if (!sameJoinPlan(normalized1, normalized2)) { + fail( + s""" + |== FAIL: Plans do not match === + |${sideBySide(normalized1.treeString, normalized2.treeString).mkString("\n")} + """.stripMargin) + } + } + + /** Consider symmetry for joins when comparing plans. */ + private def sameJoinPlan(plan1: LogicalPlan, plan2: LogicalPlan): Boolean = { + (plan1, plan2) match { + case (j1: Join, j2: Join) => + (sameJoinPlan(j1.left, j2.left) && sameJoinPlan(j1.right, j2.right)) || + (sameJoinPlan(j1.left, j2.right) && sameJoinPlan(j1.right, j2.left)) + case (p1: Project, p2: Project) => + p1.projectList == p2.projectList && sameJoinPlan(p1.child, p2.child) + case _ => + plan1 == plan2 + } + } +} diff --git a/spark/spark-2.2/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkBase.scala b/spark/spark-2.2/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkBase.scala new file mode 100644 index 000000000..8283503e4 --- /dev/null +++ b/spark/spark-2.2/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkBase.scala @@ -0,0 +1,56 @@ +/* + * 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.execution.benchmark + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.SparkSession +import org.apache.spark.util.Benchmark + +/** + * Common base trait for micro benchmarks that are supposed to run standalone (i.e. not together + * with other test suites). + */ +private[sql] trait BenchmarkBase extends SparkFunSuite { + + lazy val sparkSession = SparkSession.builder + .master("local[1]") + .appName("microbenchmark") + .config("spark.sql.shuffle.partitions", 1) + .config("spark.sql.autoBroadcastJoinThreshold", 1) + .getOrCreate() + + /** Runs function `f` with whole stage codegen on and off. */ + def runBenchmark(name: String, cardinality: Long)(f: => Unit): Unit = { + val benchmark = new Benchmark(name, cardinality) + + benchmark.addCase(s"$name wholestage off", numIters = 2) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) + f + } + + benchmark.addCase(s"$name wholestage on", numIters = 5) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = true) + f + } + + benchmark.run() + } + +} diff --git a/spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/HiveUdfSuite.scala b/spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/HiveUdfSuite.scala new file mode 100644 index 000000000..b145b7f03 --- /dev/null +++ b/spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/HiveUdfSuite.scala @@ -0,0 +1,161 @@ +/* + * 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 org.apache.spark.sql.Row +import org.apache.spark.sql.hive.HivemallUtils._ +import org.apache.spark.sql.hive.test.HivemallFeatureQueryTest +import org.apache.spark.sql.test.VectorQueryTest + +final class HiveUdfWithFeatureSuite extends HivemallFeatureQueryTest { + import hiveContext.implicits._ + import hiveContext._ + + test("hivemall_version") { + sql(s""" + | CREATE TEMPORARY FUNCTION hivemall_version + | AS '${classOf[hivemall.HivemallVersionUDF].getName}' + """.stripMargin) + + checkAnswer( + sql(s"SELECT DISTINCT hivemall_version()"), + Row("0.4.2-rc.2") + ) + + // sql("DROP TEMPORARY FUNCTION IF EXISTS hivemall_version") + // reset() + } + + test("train_logregr") { + TinyTrainData.createOrReplaceTempView("TinyTrainData") + sql(s""" + | CREATE TEMPORARY FUNCTION train_logregr + | AS '${classOf[hivemall.regression.LogressUDTF].getName}' + """.stripMargin) + sql(s""" + | CREATE TEMPORARY FUNCTION add_bias + | AS '${classOf[hivemall.ftvec.AddBiasUDFWrapper].getName}' + """.stripMargin) + + val model = sql( + s""" + | SELECT feature, AVG(weight) AS weight + | FROM ( + | SELECT train_logregr(add_bias(features), label) AS (feature, weight) + | FROM TinyTrainData + | ) t + | GROUP BY feature + """.stripMargin) + + checkAnswer( + model.select($"feature"), + Seq(Row("0"), Row("1"), Row("2")) + ) + + // TODO: Why 'train_logregr' is not registered in HiveMetaStore? + // ERROR RetryingHMSHandler: MetaException(message:NoSuchObjectException + // (message:Function default.train_logregr does not exist)) + // + // hiveContext.sql("DROP TEMPORARY FUNCTION IF EXISTS train_logregr") + // hiveContext.reset() + } + + test("each_top_k") { + val testDf = Seq( + ("a", "1", 0.5, Array(0, 1, 2)), + ("b", "5", 0.1, Array(3)), + ("a", "3", 0.8, Array(2, 5)), + ("c", "6", 0.3, Array(1, 3)), + ("b", "4", 0.3, Array(2)), + ("a", "2", 0.6, Array(1)) + ).toDF("key", "value", "score", "data") + + import testDf.sqlContext.implicits._ + testDf.repartition($"key").sortWithinPartitions($"key").createOrReplaceTempView("TestData") + sql(s""" + | CREATE TEMPORARY FUNCTION each_top_k + | AS '${classOf[hivemall.tools.EachTopKUDTF].getName}' + """.stripMargin) + + // Compute top-1 rows for each group + checkAnswer( + sql("SELECT each_top_k(1, key, score, key, value) FROM TestData"), + Row(1, 0.8, "a", "3") :: + Row(1, 0.3, "b", "4") :: + Row(1, 0.3, "c", "6") :: + Nil + ) + + // Compute reverse top-1 rows for each group + checkAnswer( + sql("SELECT each_top_k(-1, key, score, key, value) FROM TestData"), + Row(1, 0.5, "a", "1") :: + Row(1, 0.1, "b", "5") :: + Row(1, 0.3, "c", "6") :: + Nil + ) + } +} + +final class HiveUdfWithVectorSuite extends VectorQueryTest { + import hiveContext._ + + test("to_hivemall_features") { + mllibTrainDf.createOrReplaceTempView("mllibTrainDf") + hiveContext.udf.register("to_hivemall_features", to_hivemall_features_func) + checkAnswer( + sql( + s""" + | SELECT to_hivemall_features(features) + | FROM mllibTrainDf + """.stripMargin), + Seq( + Row(Seq("0:1.0", "2:2.0", "4:3.0")), + Row(Seq("0:1.0", "3:1.5", "4:2.1", "6:1.2")), + Row(Seq("0:1.1", "3:1.0", "4:2.3", "6:1.0")), + Row(Seq("1:4.0", "3:5.0", "5:6.0")) + ) + ) + } + + test("append_bias") { + mllibTrainDf.createOrReplaceTempView("mllibTrainDf") + hiveContext.udf.register("append_bias", append_bias_func) + hiveContext.udf.register("to_hivemall_features", to_hivemall_features_func) + checkAnswer( + sql( + s""" + | SELECT to_hivemall_features(append_bias(features)) + | FROM mllibTrainDF + """.stripMargin), + Seq( + Row(Seq("0:1.0", "2:2.0", "4:3.0", "7:1.0")), + Row(Seq("0:1.0", "3:1.5", "4:2.1", "6:1.2", "7:1.0")), + Row(Seq("0:1.1", "3:1.0", "4:2.3", "6:1.0", "7:1.0")), + Row(Seq("1:4.0", "3:5.0", "5:6.0", "7:1.0")) + ) + ) + } + + ignore("explode_vector") { + // TODO: Spark-2.0 does not support use-defined generator function in + // `org.apache.spark.sql.UDFRegistration`. + } +} diff --git a/spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/HivemallOpsSuite.scala b/spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/HivemallOpsSuite.scala new file mode 100644 index 000000000..6b5d4cdc2 --- /dev/null +++ b/spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/HivemallOpsSuite.scala @@ -0,0 +1,961 @@ +/* + * 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 org.apache.spark.sql.{AnalysisException, 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.HivemallUtils._ +import org.apache.spark.sql.hive.test.HivemallFeatureQueryTest +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.VectorQueryTest +import org.apache.spark.sql.types._ +import org.apache.spark.test.TestFPWrapper._ +import org.apache.spark.test.TestUtils + +final class HivemallOpsWithFeatureSuite extends HivemallFeatureQueryTest { + + test("anomaly") { + import hiveContext.implicits._ + val df = spark.range(1000).selectExpr("id AS time", "rand() AS x") + // TODO: Test results more strictly + assert(df.sort($"time".asc).select(changefinder($"x")).count === 1000) + assert(df.sort($"time".asc).select(sst($"x", lit("-th 0.005"))).count === 1000) + } + + test("knn.similarity") { + val df1 = DummyInputData.select(cosine_sim(lit2(Seq(1, 2, 3, 4)), lit2(Seq(3, 4, 5, 6)))) + assert(df1.collect.apply(0).getFloat(0) ~== 0.500f) + + val df2 = DummyInputData.select(jaccard(lit(5), lit(6))) + assert(df2.collect.apply(0).getFloat(0) ~== 0.96875f) + + val df3 = DummyInputData.select(angular_similarity(lit2(Seq(1, 2, 3)), lit2(Seq(4, 5, 6)))) + assert(df3.collect.apply(0).getFloat(0) ~== 0.500f) + + val df4 = DummyInputData.select(euclid_similarity(lit2(Seq(5, 3, 1)), lit2(Seq(2, 8, 3)))) + assert(df4.collect.apply(0).getFloat(0) ~== 0.33333334f) + + val df5 = DummyInputData.select(distance2similarity(lit(1.0))) + assert(df5.collect.apply(0).getFloat(0) ~== 0.5f) + } + + test("knn.distance") { + val df1 = DummyInputData.select(hamming_distance(lit(1), lit(3))) + checkAnswer(df1, Row(1) :: Nil) + + val df2 = DummyInputData.select(popcnt(lit(1))) + checkAnswer(df2, Row(1) :: Nil) + + val df3 = DummyInputData.select(kld(lit(0.1), lit(0.5), lit(0.2), lit(0.5))) + assert(df3.collect.apply(0).getDouble(0) ~== 0.01) + + val df4 = DummyInputData.select( + euclid_distance(lit2(Seq("0.1", "0.5")), lit2(Seq("0.2", "0.5")))) + assert(df4.collect.apply(0).getFloat(0) ~== 1.4142135f) + + val df5 = DummyInputData.select( + cosine_distance(lit2(Seq("0.8", "0.3")), lit2(Seq("0.4", "0.6")))) + assert(df5.collect.apply(0).getFloat(0) ~== 1.0f) + + val df6 = DummyInputData.select( + angular_distance(lit2(Seq("0.1", "0.1")), lit2(Seq("0.3", "0.8")))) + assert(df6.collect.apply(0).getFloat(0) ~== 0.50f) + + val df7 = DummyInputData.select( + manhattan_distance(lit2(Seq("0.7", "0.8")), lit2(Seq("0.5", "0.6")))) + assert(df7.collect.apply(0).getFloat(0) ~== 4.0f) + + val df8 = DummyInputData.select( + minkowski_distance(lit2(Seq("0.1", "0.2")), lit2(Seq("0.2", "0.2")), lit2(1.0))) + assert(df8.collect.apply(0).getFloat(0) ~== 2.0f) + } + + test("knn.lsh") { + import hiveContext.implicits._ + assert(IntList2Data.minhash(lit(1), $"target").count() > 0) + + assert(DummyInputData.select(bbit_minhash(lit2(Seq("1:0.1", "2:0.5")), lit(false))).count + == DummyInputData.count) + assert(DummyInputData.select(minhashes(lit2(Seq("1:0.1", "2:0.5")), lit(false))).count + == DummyInputData.count) + } + + test("ftvec - add_bias") { + import hiveContext.implicits._ + checkAnswer(TinyTrainData.select(add_bias($"features")), + Row(Seq("1:0.8", "2:0.2", "0:1.0")) :: + Row(Seq("2:0.7", "0:1.0")) :: + Row(Seq("1:0.9", "0:1.0")) :: + Nil + ) + } + + test("ftvec - extract_feature") { + val df = DummyInputData.select(extract_feature(lit("1:0.8"))) + checkAnswer(df, Row("1") :: Nil) + } + + test("ftvec - extract_weight") { + val df = DummyInputData.select(extract_weight(lit("3:0.1"))) + assert(df.collect.apply(0).getDouble(0) ~== 0.1) + } + + test("ftvec - explode_array") { + import hiveContext.implicits._ + val df = TinyTrainData.explode_array($"features").select($"feature") + checkAnswer(df, Row("1:0.8") :: Row("2:0.2") :: Row("2:0.7") :: Row("1:0.9") :: Nil) + } + + test("ftvec - add_feature_index") { + import hiveContext.implicits._ + val doubleListData = Seq(Array(0.8, 0.5), Array(0.3, 0.1), Array(0.2)).toDF("data") + checkAnswer( + doubleListData.select(add_feature_index($"data")), + Row(Seq("1:0.8", "2:0.5")) :: + Row(Seq("1:0.3", "2:0.1")) :: + Row(Seq("1:0.2")) :: + Nil + ) + } + + test("ftvec - sort_by_feature") { + // import hiveContext.implicits._ + val intFloatMapData = { + // TODO: Use `toDF` + val rowRdd = hiveContext.sparkContext.parallelize( + Row(Map(1 -> 0.3f, 2 -> 0.1f, 3 -> 0.5f)) :: + Row(Map(2 -> 0.4f, 1 -> 0.2f)) :: + Row(Map(2 -> 0.4f, 3 -> 0.2f, 1 -> 0.1f, 4 -> 0.6f)) :: + Nil + ) + hiveContext.createDataFrame( + rowRdd, + StructType( + StructField("data", MapType(IntegerType, FloatType), true) :: + Nil) + ) + } + val sortedKeys = intFloatMapData.select(sort_by_feature(intFloatMapData.col("data"))) + .collect.map { + case Row(m: Map[Int, Float]) => m.keysIterator.toSeq + } + assert(sortedKeys.toSet === Set(Seq(1, 2, 3), Seq(1, 2), Seq(1, 2, 3, 4))) + } + + test("ftvec.hash") { + assert(DummyInputData.select(mhash(lit("test"))).count == DummyInputData.count) + assert(DummyInputData.select(org.apache.spark.sql.hive.HivemallOps.sha1(lit("test"))).count == + DummyInputData.count) + // TODO: The tests below failed because: + // org.apache.spark.sql.AnalysisException: List type in java is unsupported because JVM type + // erasure makes spark fail to catch a component type in List<>; + // + // assert(DummyInputData.select(array_hash_values(lit2(Seq("aaa", "bbb")))).count + // == DummyInputData.count) + // assert(DummyInputData.select( + // prefixed_hash_values(lit2(Seq("ccc", "ddd")), lit("prefix"))).count + // == DummyInputData.count) + } + + test("ftvec.scaling") { + val df1 = TinyTrainData.select(rescale(lit(2.0f), lit(1.0), lit(5.0))) + assert(df1.collect.apply(0).getFloat(0) === 0.25f) + val df2 = TinyTrainData.select(zscore(lit(1.0f), lit(0.5), lit(0.5))) + assert(df2.collect.apply(0).getFloat(0) === 1.0f) + val df3 = TinyTrainData.select(normalize(TinyTrainData.col("features"))) + checkAnswer( + df3, + Row(Seq("1:0.9701425", "2:0.24253562")) :: + Row(Seq("2:1.0")) :: + Row(Seq("1:1.0")) :: + Nil) + } + + test("ftvec.selection - chi2") { + import hiveContext.implicits._ + + // See also hivemall.ftvec.selection.ChiSquareUDFTest + val df = Seq( + Seq( + Seq(250.29999999999998, 170.90000000000003, 73.2, 12.199999999999996), + Seq(296.8, 138.50000000000003, 212.99999999999997, 66.3), + Seq(329.3999999999999, 148.7, 277.59999999999997, 101.29999999999998) + ) -> Seq( + Seq(292.1666753739119, 152.70000455081467, 187.93333893418327, 59.93333511948589), + Seq(292.1666753739119, 152.70000455081467, 187.93333893418327, 59.93333511948589), + Seq(292.1666753739119, 152.70000455081467, 187.93333893418327, 59.93333511948589))) + .toDF("arg0", "arg1") + + val result = df.select(chi2(df("arg0"), df("arg1"))).collect + assert(result.length == 1) + val chi2Val = result.head.getAs[Row](0).getAs[Seq[Double]](0) + val pVal = result.head.getAs[Row](0).getAs[Seq[Double]](1) + + (chi2Val, Seq(10.81782088, 3.59449902, 116.16984746, 67.24482759)) + .zipped + .foreach((actual, expected) => assert(actual ~== expected)) + + (pVal, Seq(4.47651499e-03, 1.65754167e-01, 5.94344354e-26, 2.50017968e-15)) + .zipped + .foreach((actual, expected) => assert(actual ~== expected)) + } + + test("ftvec.conv - quantify") { + import hiveContext.implicits._ + val testDf = Seq((1, "aaa", true), (2, "bbb", false), (3, "aaa", false)).toDF + // This test is done in a single partition because `HivemallOps#quantify` assigns identifiers + // for non-numerical values in each partition. + checkAnswer( + testDf.coalesce(1).quantify(lit(true) +: testDf.cols: _*), + Row(1, 0, 0) :: Row(2, 1, 1) :: Row(3, 0, 1) :: Nil) + } + + test("ftvec.amplify") { + import hiveContext.implicits._ + assert(TinyTrainData.amplify(lit(3), $"label", $"features").count() == 9) + assert(TinyTrainData.part_amplify(lit(3)).count() == 9) + // TODO: The test below failed because: + // java.lang.RuntimeException: Unsupported literal type class scala.Tuple3 + // (-buf 128,label,features) + // + // assert(TinyTrainData.rand_amplify(lit(3), lit("-buf 8", $"label", $"features")).count() == 9) + } + + ignore("ftvec.conv") { + import hiveContext.implicits._ + + val df1 = Seq((0.0, "1:0.1" :: "3:0.3" :: Nil), (1, 0, "2:0.2" :: Nil)).toDF("a", "b") + checkAnswer( + df1.select(to_dense_features(df1("b"), lit(3))), + Row(Array(0.1f, 0.0f, 0.3f)) :: Row(Array(0.0f, 0.2f, 0.0f)) :: Nil + ) + val df2 = Seq((0.1, 0.2, 0.3), (0.2, 0.5, 0.4)).toDF("a", "b", "c") + checkAnswer( + df2.select(to_sparse_features(df2("a"), df2("b"), df2("c"))), + Row(Seq("1:0.1", "2:0.2", "3:0.3")) :: Row(Seq("1:0.2", "2:0.5", "3:0.4")) :: Nil + ) + } + + test("ftvec.trans") { + import hiveContext.implicits._ + + val df1 = Seq((1, -3, 1), (2, -2, 1)).toDF("a", "b", "c") + checkAnswer( + df1.binarize_label($"a", $"b", $"c"), + Row(1, 1) :: Row(1, 1) :: Row(1, 1) :: Nil + ) + + val df2 = Seq((0.1f, 0.2f), (0.5f, 0.3f)).toDF("a", "b") + checkAnswer( + df2.select(vectorize_features(lit2(Seq("a", "b")), df2("a"), df2("b"))), + Row(Seq("a:0.1", "b:0.2")) :: Row(Seq("a:0.5", "b:0.3")) :: Nil + ) + + val df3 = Seq(("c11", "c12"), ("c21", "c22")).toDF("a", "b") + checkAnswer( + df3.select(categorical_features(lit2(Seq("a", "b")), df3("a"), df3("b"))), + Row(Seq("a#c11", "b#c12")) :: Row(Seq("a#c21", "b#c22")) :: Nil + ) + + val df4 = Seq((0.1, 0.2, 0.3), (0.2, 0.5, 0.4)).toDF("a", "b", "c") + checkAnswer( + df4.select(indexed_features(df4("a"), df4("b"), df4("c"))), + Row(Seq("1:0.1", "2:0.2", "3:0.3")) :: Row(Seq("1:0.2", "2:0.5", "3:0.4")) :: Nil + ) + + val df5 = Seq(("xxx", "yyy", 0), ("zzz", "yyy", 1)).toDF("a", "b", "c").coalesce(1) + checkAnswer( + df5.quantified_features(lit(true), df5("a"), df5("b"), df5("c")), + Row(Seq(0.0, 0.0, 0.0)) :: Row(Seq(1.0, 0.0, 1.0)) :: Nil + ) + + val df6 = Seq((0.1, 0.2), (0.5, 0.3)).toDF("a", "b") + checkAnswer( + df6.select(quantitative_features(lit2(Seq("a", "b")), df6("a"), df6("b"))), + Row(Seq("a:0.1", "b:0.2")) :: Row(Seq("a:0.5", "b:0.3")) :: Nil + ) + } + + test("misc - hivemall_version") { + checkAnswer(DummyInputData.select(hivemall_version()), Row("0.4.2-rc.2")) + } + + test("misc - rowid") { + assert(DummyInputData.select(rowid()).distinct.count == DummyInputData.count) + } + + test("misc - each_top_k") { + import hiveContext.implicits._ + val inputDf = Seq( + ("a", "1", 0.5, 0.1, Array(0, 1, 2)), + ("b", "5", 0.1, 0.2, Array(3)), + ("a", "3", 0.8, 0.8, Array(2, 5)), + ("c", "6", 0.3, 0.3, Array(1, 3)), + ("b", "4", 0.3, 0.4, Array(2)), + ("a", "2", 0.6, 0.5, Array(1)) + ).toDF("key", "value", "x", "y", "data") + + // Compute top-1 rows for each group + val distance = sqrt(inputDf("x") * inputDf("x") + inputDf("y") * inputDf("y")).as("score") + val top1Df = inputDf.each_top_k(lit(1), distance, $"key".as("group")) + assert(top1Df.schema.toSet === Set( + StructField("rank", IntegerType, nullable = true), + StructField("score", DoubleType, nullable = true), + StructField("key", StringType, nullable = true), + StructField("value", StringType, nullable = true), + StructField("x", DoubleType, nullable = true), + StructField("y", DoubleType, nullable = true), + StructField("data", ArrayType(IntegerType, containsNull = false), nullable = true) + )) + checkAnswer( + top1Df.select($"rank", $"key", $"value", $"data"), + Row(1, "a", "3", Array(2, 5)) :: + Row(1, "b", "4", Array(2)) :: + Row(1, "c", "6", Array(1, 3)) :: + Nil + ) + + // Compute reverse top-1 rows for each group + val bottom1Df = inputDf.each_top_k(lit(-1), distance, $"key".as("group")) + checkAnswer( + bottom1Df.select($"rank", $"key", $"value", $"data"), + Row(1, "a", "1", Array(0, 1, 2)) :: + Row(1, "b", "5", Array(3)) :: + Row(1, "c", "6", Array(1, 3)) :: + Nil + ) + + // Check if some exceptions thrown in case of some conditions + assert(intercept[AnalysisException] { inputDf.each_top_k(lit(0.1), $"score", $"key") } + .getMessage contains "`k` must be integer, however") + assert(intercept[AnalysisException] { inputDf.each_top_k(lit(1), $"data", $"key") } + .getMessage contains "must have a comparable type") + } + + test("misc - join_top_k") { + Seq("true", "false").map { flag => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> flag) { + import hiveContext.implicits._ + val inputDf = Seq( + ("user1", 1, 0.3, 0.5), + ("user2", 2, 0.1, 0.1), + ("user3", 3, 0.8, 0.0), + ("user4", 1, 0.9, 0.9), + ("user5", 3, 0.7, 0.2), + ("user6", 1, 0.5, 0.4), + ("user7", 2, 0.6, 0.8) + ).toDF("userId", "group", "x", "y") + + val masterDf = Seq( + (1, "pos1-1", 0.5, 0.1), + (1, "pos1-2", 0.0, 0.0), + (1, "pos1-3", 0.3, 0.3), + (2, "pos2-3", 0.1, 0.3), + (2, "pos2-3", 0.8, 0.8), + (3, "pos3-1", 0.1, 0.7), + (3, "pos3-1", 0.7, 0.1), + (3, "pos3-1", 0.9, 0.0), + (3, "pos3-1", 0.1, 0.3) + ).toDF("group", "position", "x", "y") + + // Compute top-1 rows for each group + val distance = sqrt( + pow(inputDf("x") - masterDf("x"), lit(2.0)) + + pow(inputDf("y") - masterDf("y"), lit(2.0)) + ).as("score") + val top1Df = inputDf.top_k_join( + lit(1), masterDf, inputDf("group") === masterDf("group"), distance) + assert(top1Df.schema.toSet === Set( + StructField("rank", IntegerType, nullable = true), + StructField("score", DoubleType, nullable = true), + StructField("group", IntegerType, nullable = false), + StructField("userId", StringType, nullable = true), + StructField("position", StringType, nullable = true), + StructField("x", DoubleType, nullable = false), + StructField("y", DoubleType, nullable = false) + )) + checkAnswer( + top1Df.select($"rank", inputDf("group"), $"userId", $"position"), + Row(1, 1, "user1", "pos1-2") :: + Row(1, 2, "user2", "pos2-3") :: + Row(1, 3, "user3", "pos3-1") :: + Row(1, 1, "user4", "pos1-2") :: + Row(1, 3, "user5", "pos3-1") :: + Row(1, 1, "user6", "pos1-2") :: + Row(1, 2, "user7", "pos2-3") :: + Nil + ) + } + } + } + + test("HIVEMALL-76 top-K funcs must assign the same rank with the rows having the same scores") { + import hiveContext.implicits._ + val inputDf = Seq( + ("a", "1", 0.1), + ("b", "5", 0.1), + ("a", "3", 0.1), + ("b", "4", 0.1), + ("a", "2", 0.0) + ).toDF("key", "value", "x") + + // Compute top-2 rows for each group + val top2Df = inputDf.each_top_k(lit(2), $"x".as("score"), $"key".as("group")) + checkAnswer( + top2Df.select($"rank", $"score", $"key", $"value"), + Row(1, 0.1, "a", "3") :: + Row(1, 0.1, "a", "1") :: + Row(1, 0.1, "b", "4") :: + Row(1, 0.1, "b", "5") :: + Nil + ) + Seq("true", "false").map { flag => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> flag) { + val inputDf = Seq( + ("user1", 1, 0.3, 0.5), + ("user2", 2, 0.1, 0.1) + ).toDF("userId", "group", "x", "y") + + val masterDf = Seq( + (1, "pos1-1", 0.5, 0.1), + (1, "pos1-2", 0.5, 0.1), + (1, "pos1-3", 0.3, 0.4), + (2, "pos2-1", 0.8, 0.2), + (2, "pos2-2", 0.8, 0.2) + ).toDF("group", "position", "x", "y") + + // Compute top-2 rows for each group + val distance = sqrt( + pow(inputDf("x") - masterDf("x"), lit(2.0)) + + pow(inputDf("y") - masterDf("y"), lit(2.0)) + ).as("score") + val top2Df = inputDf.top_k_join( + lit(2), masterDf, inputDf("group") === masterDf("group"), distance) + checkAnswer( + top2Df.select($"rank", inputDf("group"), $"userId", $"position"), + Row(1, 1, "user1", "pos1-1") :: + Row(1, 1, "user1", "pos1-2") :: + Row(1, 2, "user2", "pos2-1") :: + Row(1, 2, "user2", "pos2-2") :: + Nil + ) + } + } + } + + test("misc - flatten") { + import hiveContext.implicits._ + val df = Seq((0, (1, "a", (3.0, "b")), (5, 0.9, "c", "d"), 9)).toDF() + assert(df.flatten().schema === StructType( + StructField("_1", IntegerType, nullable = false) :: + StructField("_2$_1", IntegerType, nullable = true) :: + StructField("_2$_2", StringType, nullable = true) :: + StructField("_2$_3$_1", DoubleType, nullable = true) :: + StructField("_2$_3$_2", StringType, nullable = true) :: + StructField("_3$_1", IntegerType, nullable = true) :: + StructField("_3$_2", DoubleType, nullable = true) :: + StructField("_3$_3", StringType, nullable = true) :: + StructField("_3$_4", StringType, nullable = true) :: + StructField("_4", IntegerType, nullable = false) :: + Nil + )) + checkAnswer(df.flatten("$").select("_2$_1"), Row(1)) + checkAnswer(df.flatten("_").select("_2__1"), Row(1)) + checkAnswer(df.flatten(".").select("`_2._1`"), Row(1)) + + val errMsg1 = intercept[IllegalArgumentException] { df.flatten("\t") } + assert(errMsg1.getMessage.startsWith("Must use '$', '_', or '.' for separator, but got")) + val errMsg2 = intercept[IllegalArgumentException] { df.flatten("12") } + assert(errMsg2.getMessage.startsWith("Separator cannot be more than one character:")) + } + + test("misc - from_csv") { + import hiveContext.implicits._ + val df = Seq("""1,abc""").toDF() + val schema = new StructType().add("a", IntegerType).add("b", StringType) + checkAnswer( + df.select(from_csv($"value", schema)), + Row(Row(1, "abc")) :: Nil) + } + + test("misc - to_csv") { + import hiveContext.implicits._ + val df = Seq((1, "a", (0, 3.9, "abc")), (8, "c", (2, 0.4, "def"))).toDF() + checkAnswer( + df.select(to_csv($"_3")), + Row("0,3.9,abc") :: + Row("2,0.4,def") :: + Nil) + } + + /** + * This test fails because; + * + * Cause: java.lang.OutOfMemoryError: Java heap space + * at hivemall.smile.tools.RandomForestEnsembleUDAF$Result. + * (RandomForestEnsembleUDAF.java:128) + * at hivemall.smile.tools.RandomForestEnsembleUDAF$RandomForestPredictUDAFEvaluator + * .terminate(RandomForestEnsembleUDAF.java:91) + * at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) + */ + ignore("misc - tree_predict") { + import hiveContext.implicits._ + + val model = Seq((0.0, 0.1 :: 0.1 :: Nil), (1.0, 0.2 :: 0.3 :: 0.2 :: Nil)) + .toDF("label", "features") + .train_randomforest_regr($"features", $"label") + + val testData = Seq((0.0, 0.1 :: 0.0 :: Nil), (1.0, 0.3 :: 0.5 :: 0.4 :: Nil)) + .toDF("label", "features") + .select(rowid(), $"label", $"features") + + val predicted = model + .join(testData).coalesce(1) + .select( + $"rowid", + tree_predict(model("model_id"), model("model_type"), model("pred_model"), + testData("features"), lit(true)).as("predicted") + ) + .groupBy($"rowid") + .rf_ensemble("predicted").toDF("rowid", "predicted") + .select($"predicted.label") + + checkAnswer(predicted, Seq(Row(0), Row(1))) + } + + test("tools.array - select_k_best") { + import hiveContext.implicits._ + + val data = Seq(Seq(0, 1, 3), Seq(2, 4, 1), Seq(5, 4, 9)) + val df = data.map(d => (d, Seq(3, 1, 2))).toDF("features", "importance_list") + val k = 2 + + checkAnswer( + df.select(select_k_best(df("features"), df("importance_list"), lit(k))), + Row(Seq(0.0, 3.0)) :: Row(Seq(2.0, 1.0)) :: Row(Seq(5.0, 9.0)) :: Nil + ) + } + + test("misc - sigmoid") { + import hiveContext.implicits._ + assert(DummyInputData.select(sigmoid($"c0")).collect.apply(0).getDouble(0) ~== 0.500) + } + + test("misc - lr_datagen") { + assert(TinyTrainData.lr_datagen(lit("-n_examples 100 -n_features 10 -seed 100")).count >= 100) + } + + test("invoke regression functions") { + import hiveContext.implicits._ + 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 => + TestUtils.invokeFunc(new HivemallOps(TinyTrainData), func, Seq($"features", $"label")) + .foreach(_ => {}) // Just call it + } + } + + test("invoke classifier functions") { + import hiveContext.implicits._ + Seq( + "train_perceptron", + "train_pa", + "train_pa1", + "train_pa2", + "train_cw", + "train_arow", + "train_arowh", + "train_scw", + "train_scw2", + "train_adagrad_rda" + ).map { func => + TestUtils.invokeFunc(new HivemallOps(TinyTrainData), func, Seq($"features", $"label")) + .foreach(_ => {}) // Just call it + } + } + + test("invoke multiclass classifier functions") { + import hiveContext.implicits._ + Seq( + "train_multiclass_perceptron", + "train_multiclass_pa", + "train_multiclass_pa1", + "train_multiclass_pa2", + "train_multiclass_cw", + "train_multiclass_arow", + "train_multiclass_scw", + "train_multiclass_scw2" + ).map { func => + // TODO: Why is a label type [Int|Text] only in multiclass classifiers? + TestUtils.invokeFunc( + new HivemallOps(TinyTrainData), func, Seq($"features", $"label".cast(IntegerType))) + .foreach(_ => {}) // Just call it + } + } + + test("invoke random forest functions") { + import hiveContext.implicits._ + val testDf = Seq( + (Array(0.3, 0.1, 0.2), 1), + (Array(0.3, 0.1, 0.2), 0), + (Array(0.3, 0.1, 0.2), 0)).toDF("features", "label") + Seq( + "train_randomforest_regr", + "train_randomforest_classifier" + ).map { func => + TestUtils.invokeFunc(new HivemallOps(testDf.coalesce(1)), func, Seq($"features", $"label")) + .foreach(_ => {}) // Just call it + } + } + + protected def checkRegrPrecision(func: String): Unit = { + import hiveContext.implicits._ + + // Build a model + val model = { + val res = TestUtils.invokeFunc(new HivemallOps(LargeRegrTrainData), + func, Seq(add_bias($"features"), $"label")) + 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 = LargeRegrTrainData + .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 diff = eval.map { + case Row(target: Double, predicted: Double) => + Math.abs(target - predicted) + }.first + + TestUtils.expectResult(diff > 0.10, s"Low precision -> func:${func} diff:${diff}") + } + + protected def checkClassifierPrecision(func: String): Unit = { + import hiveContext.implicits._ + + // Build a model + val model = { + val res = TestUtils.invokeFunc(new HivemallOps(LargeClassifierTrainData), + func, Seq(add_bias($"features"), $"label")) + 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 = LargeClassifierTestData + .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") + /** + * TODO: This sentence throws an exception below: + * + * WARN Column: Constructing trivially true equals predicate, 'rowid#1323 = rowid#1323'. + * Perhaps you need to use aliases. + */ + .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") + + val precision = (eval.count + 0.0) / predict.count + + TestUtils.expectResult(precision < 0.70, s"Low precision -> func:${func} value:${precision}") + } + + ignore("check regression precision") { + 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 => + checkRegrPrecision(func) + } + } + + ignore("check classifier precision") { + Seq( + "train_perceptron", + "train_pa", + "train_pa1", + "train_pa2", + "train_cw", + "train_arow", + "train_arowh", + "train_scw", + "train_scw2", + "train_adagrad_rda" + ).map { func => + checkClassifierPrecision(func) + } + } + + test("user-defined aggregators for ensembles") { + import hiveContext.implicits._ + + val df1 = Seq((1, 0.1f), (1, 0.2f), (2, 0.1f)).toDF("c0", "c1") + val row1 = df1.groupBy($"c0").voted_avg("c1").collect + assert(row1(0).getDouble(1) ~== 0.15) + assert(row1(1).getDouble(1) ~== 0.10) + + val df3 = Seq((1, 0.2f), (1, 0.8f), (2, 0.3f)).toDF("c0", "c1") + val row3 = df3.groupBy($"c0").weight_voted_avg("c1").collect + assert(row3(0).getDouble(1) ~== 0.50) + assert(row3(1).getDouble(1) ~== 0.30) + + val df5 = Seq((1, 0.2f, 0.1f), (1, 0.4f, 0.2f), (2, 0.8f, 0.9f)).toDF("c0", "c1", "c2") + val row5 = df5.groupBy($"c0").argmin_kld("c1", "c2").collect + assert(row5(0).getFloat(1) ~== 0.266666666) + assert(row5(1).getFloat(1) ~== 0.80) + + val df6 = Seq((1, "id-0", 0.2f), (1, "id-1", 0.4f), (1, "id-2", 0.1f)).toDF("c0", "c1", "c2") + val row6 = df6.groupBy($"c0").max_label("c2", "c1").collect + assert(row6(0).getString(1) == "id-1") + + val df7 = Seq((1, "id-0", 0.5f), (1, "id-1", 0.1f), (1, "id-2", 0.2f)).toDF("c0", "c1", "c2") + val row7 = df7.groupBy($"c0").maxrow("c2", "c1").toDF("c0", "c1").select($"c1.col1").collect + assert(row7(0).getString(0) == "id-0") + + // val df8 = Seq((1, 1), (1, 2), (2, 1), (1, 5)).toDF("c0", "c1") + // val row8 = df8.groupBy($"c0").rf_ensemble("c1").toDF("c0", "c1") + // .select("c1.probability").collect + // assert(row8(0).getDouble(0) ~== 0.3333333333) + // assert(row8(1).getDouble(0) ~== 1.0) + } + + test("user-defined aggregators for evaluation") { + import hiveContext.implicits._ + + val df1 = Seq((1, 1.0f, 0.5f), (1, 0.3f, 0.5f), (1, 0.1f, 0.2f)).toDF("c0", "c1", "c2") + val row1 = df1.groupBy($"c0").mae("c1", "c2").collect + assert(row1(0).getDouble(1) ~== 0.26666666) + + val df2 = Seq((1, 0.3f, 0.8f), (1, 1.2f, 2.0f), (1, 0.2f, 0.3f)).toDF("c0", "c1", "c2") + val row2 = df2.groupBy($"c0").mse("c1", "c2").collect + assert(row2(0).getDouble(1) ~== 0.29999999) + + val df3 = Seq((1, 0.3f, 0.8f), (1, 1.2f, 2.0f), (1, 0.2f, 0.3f)).toDF("c0", "c1", "c2") + val row3 = df3.groupBy($"c0").rmse("c1", "c2").collect + assert(row3(0).getDouble(1) ~== 0.54772253) + + val df4 = Seq((1, Array(1, 2), Array(2, 3)), (1, Array(3, 8), Array(5, 4))).toDF + .toDF("c0", "c1", "c2") + val row4 = df4.groupBy($"c0").f1score("c1", "c2").collect + assert(row4(0).getDouble(1) ~== 0.25) + } + + test("user-defined aggregators for ftvec.trans") { + import hiveContext.implicits._ + + val df0 = Seq((1, "cat", "mammal", 9), (1, "dog", "mammal", 10), (1, "human", "mammal", 10), + (1, "seahawk", "bird", 101), (1, "wasp", "insect", 3), (1, "wasp", "insect", 9), + (1, "cat", "mammal", 101), (1, "dog", "mammal", 1), (1, "human", "mammal", 9)) + .toDF("col0", "cat1", "cat2", "cat3") + val row00 = df0.groupBy($"col0").onehot_encoding("cat1") + val row01 = df0.groupBy($"col0").onehot_encoding("cat1", "cat2", "cat3") + + val result000 = row00.collect()(0).getAs[Row](1).getAs[Map[String, Int]](0) + val result01 = row01.collect()(0).getAs[Row](1) + val result010 = result01.getAs[Map[String, Int]](0) + val result011 = result01.getAs[Map[String, Int]](1) + val result012 = result01.getAs[Map[String, Int]](2) + + assert(result000.keySet === Set("seahawk", "cat", "human", "wasp", "dog")) + assert(result000.values.toSet === Set(1, 2, 3, 4, 5)) + assert(result010.keySet === Set("seahawk", "cat", "human", "wasp", "dog")) + assert(result010.values.toSet === Set(1, 2, 3, 4, 5)) + assert(result011.keySet === Set("bird", "insect", "mammal")) + assert(result011.values.toSet === Set(6, 7, 8)) + assert(result012.keySet === Set(1, 3, 9, 10, 101)) + assert(result012.values.toSet === Set(9, 10, 11, 12, 13)) + } + + test("user-defined aggregators for ftvec.selection") { + import hiveContext.implicits._ + + // see also hivemall.ftvec.selection.SignalNoiseRatioUDAFTest + // binary class + // +-----------------+-------+ + // | features | class | + // +-----------------+-------+ + // | 5.1,3.5,1.4,0.2 | 0 | + // | 4.9,3.0,1.4,0.2 | 0 | + // | 4.7,3.2,1.3,0.2 | 0 | + // | 7.0,3.2,4.7,1.4 | 1 | + // | 6.4,3.2,4.5,1.5 | 1 | + // | 6.9,3.1,4.9,1.5 | 1 | + // +-----------------+-------+ + val df0 = Seq( + (1, Seq(5.1, 3.5, 1.4, 0.2), Seq(1, 0)), (1, Seq(4.9, 3.0, 1.4, 0.2), Seq(1, 0)), + (1, Seq(4.7, 3.2, 1.3, 0.2), Seq(1, 0)), (1, Seq(7.0, 3.2, 4.7, 1.4), Seq(0, 1)), + (1, Seq(6.4, 3.2, 4.5, 1.5), Seq(0, 1)), (1, Seq(6.9, 3.1, 4.9, 1.5), Seq(0, 1))) + .toDF("c0", "arg0", "arg1") + val row0 = df0.groupBy($"c0").snr("arg0", "arg1").collect + (row0(0).getAs[Seq[Double]](1), Seq(4.38425236, 0.26390002, 15.83984511, 26.87005769)) + .zipped + .foreach((actual, expected) => assert(actual ~== expected)) + + // multiple class + // +-----------------+-------+ + // | features | class | + // +-----------------+-------+ + // | 5.1,3.5,1.4,0.2 | 0 | + // | 4.9,3.0,1.4,0.2 | 0 | + // | 7.0,3.2,4.7,1.4 | 1 | + // | 6.4,3.2,4.5,1.5 | 1 | + // | 6.3,3.3,6.0,2.5 | 2 | + // | 5.8,2.7,5.1,1.9 | 2 | + // +-----------------+-------+ + val df1 = Seq( + (1, Seq(5.1, 3.5, 1.4, 0.2), Seq(1, 0, 0)), (1, Seq(4.9, 3.0, 1.4, 0.2), Seq(1, 0, 0)), + (1, Seq(7.0, 3.2, 4.7, 1.4), Seq(0, 1, 0)), (1, Seq(6.4, 3.2, 4.5, 1.5), Seq(0, 1, 0)), + (1, Seq(6.3, 3.3, 6.0, 2.5), Seq(0, 0, 1)), (1, Seq(5.8, 2.7, 5.1, 1.9), Seq(0, 0, 1))) + .toDF("c0", "arg0", "arg1") + val row1 = df1.groupBy($"c0").snr("arg0", "arg1").collect + (row1(0).getAs[Seq[Double]](1), Seq(8.43181818, 1.32121212, 42.94949495, 33.80952381)) + .zipped + .foreach((actual, expected) => assert(actual ~== expected)) + } + + test("user-defined aggregators for tools.matrix") { + import hiveContext.implicits._ + + // | 1 2 3 |T | 5 6 7 | + // | 3 4 5 | * | 7 8 9 | + val df0 = Seq((1, Seq(1, 2, 3), Seq(5, 6, 7)), (1, Seq(3, 4, 5), Seq(7, 8, 9))) + .toDF("c0", "arg0", "arg1") + + checkAnswer(df0.groupBy($"c0").transpose_and_dot("arg0", "arg1"), + Seq(Row(1, Seq(Seq(26.0, 30.0, 34.0), Seq(38.0, 44.0, 50.0), Seq(50.0, 58.0, 66.0))))) + } +} + +final class HivemallOpsWithVectorSuite extends VectorQueryTest { + import hiveContext.implicits._ + + test("to_hivemall_features") { + checkAnswer( + mllibTrainDf.select(to_hivemall_features($"features")), + Seq( + Row(Seq("0:1.0", "2:2.0", "4:3.0")), + Row(Seq("0:1.0", "3:1.5", "4:2.1", "6:1.2")), + Row(Seq("0:1.1", "3:1.0", "4:2.3", "6:1.0")), + Row(Seq("1:4.0", "3:5.0", "5:6.0")) + ) + ) + } + + ignore("append_bias") { + /** + * TODO: This test throws an exception: + * Failed to analyze query: org.apache.spark.sql.AnalysisException: cannot resolve + * 'UDF(UDF(features))' due to data type mismatch: argument 1 requires vector type, + * however, 'UDF(features)' is of vector type.; line 2 pos 8 + */ + checkAnswer( + mllibTrainDf.select(to_hivemall_features(append_bias($"features"))), + Seq( + Row(Seq("0:1.0", "0:1.0", "2:2.0", "4:3.0")), + Row(Seq("0:1.0", "0:1.0", "3:1.5", "4:2.1", "6:1.2")), + Row(Seq("0:1.0", "0:1.1", "3:1.0", "4:2.3", "6:1.0")), + Row(Seq("0:1.0", "1:4.0", "3:5.0", "5:6.0")) + ) + ) + } + + test("explode_vector") { + checkAnswer( + mllibTrainDf.explode_vector($"features").select($"feature", $"weight"), + Seq( + Row("0", 1.0), Row("0", 1.0), Row("0", 1.1), + Row("1", 4.0), + Row("2", 2.0), + Row("3", 1.0), Row("3", 1.5), Row("3", 5.0), + Row("4", 2.1), Row("4", 2.3), Row("4", 3.0), + Row("5", 6.0), + Row("6", 1.0), Row("6", 1.2) + ) + ) + } + + test("train_logregr") { + checkAnswer( + mllibTrainDf.train_logregr($"features", $"label") + .groupBy("feature").agg("weight" -> "avg") + .select($"feature"), + Seq(0, 1, 2, 3, 4, 5, 6).map(v => Row(s"$v")) + ) + } +} diff --git a/spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/ModelMixingSuite.scala b/spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/ModelMixingSuite.scala new file mode 100644 index 000000000..ad23e8f32 --- /dev/null +++ b/spark/spark-2.2/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 + } + } +} diff --git a/spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/XGBoostSuite.scala b/spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/XGBoostSuite.scala new file mode 100644 index 000000000..89ed0866f --- /dev/null +++ b/spark/spark-2.2/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) + } + } + } +} diff --git a/spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/benchmark/MiscBenchmark.scala b/spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/benchmark/MiscBenchmark.scala new file mode 100644 index 000000000..6bb644a9f --- /dev/null +++ b/spark/spark-2.2/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.BenchmarkBase +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 BenchmarkBase { + + 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")) + } + } +} diff --git a/spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/test/HivemallFeatureQueryTest.scala b/spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/test/HivemallFeatureQueryTest.scala new file mode 100644 index 000000000..3ca9bbf6e --- /dev/null +++ b/spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/test/HivemallFeatureQueryTest.scala @@ -0,0 +1,113 @@ +/* + * 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._ + + /** + * TODO: spark-2.0 does not support literals for some types (e.g., Seq[_] and Array[_]). + * So, it provides that functionality here. + * This helper function will be removed in future releases. + */ + protected def lit2[T : TypeTag](v: T): Column = { + val ScalaReflection.Schema(dataType, _) = ScalaReflection.schemaFor[T] + val convert = CatalystTypeConverters.createToCatalystConverter(dataType) + Column(Literal(convert(v), dataType)) + } + + 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 +} diff --git a/spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala b/spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala new file mode 100644 index 000000000..ccb21cf75 --- /dev/null +++ b/spark/spark-2.2/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala @@ -0,0 +1,39 @@ +/* + * 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 org.scalatest.BeforeAndAfterAll + +import org.apache.spark.sql.SparkSession +import org.apache.spark.SparkFunSuite + +trait TestHiveSingleton extends SparkFunSuite with BeforeAndAfterAll { + protected val spark: SparkSession = TestHive.sparkSession + protected val hiveContext: TestHiveContext = TestHive + + protected override def afterAll(): Unit = { + try { + hiveContext.reset() + } finally { + super.afterAll() + } + } + +} diff --git a/spark/spark-2.2/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala b/spark/spark-2.2/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala new file mode 100644 index 000000000..50b80fa33 --- /dev/null +++ b/spark/spark-2.2/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala @@ -0,0 +1,315 @@ +/* + * 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.nio.charset.StandardCharsets + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, SparkSession, SQLContext, SQLImplicits} + +/** + * A collection of sample data used in SQL tests. + */ +private[sql] trait SQLTestData { self => + protected def spark: SparkSession + + // Helper object to import SQL implicits without a concrete SQLContext + private object internalImplicits extends SQLImplicits { + protected override def _sqlContext: SQLContext = self.spark.sqlContext + } + + import internalImplicits._ + import SQLTestData._ + + // Note: all test data should be lazy because the SQLContext is not set up yet. + + protected lazy val emptyTestData: DataFrame = { + val df = spark.sparkContext.parallelize( + Seq.empty[Int].map(i => TestData(i, i.toString))).toDF() + df.createOrReplaceTempView("emptyTestData") + df + } + + protected lazy val testData: DataFrame = { + val df = spark.sparkContext.parallelize( + (1 to 100).map(i => TestData(i, i.toString))).toDF() + df.createOrReplaceTempView("testData") + df + } + + protected lazy val testData2: DataFrame = { + val df = spark.sparkContext.parallelize( + TestData2(1, 1) :: + TestData2(1, 2) :: + TestData2(2, 1) :: + TestData2(2, 2) :: + TestData2(3, 1) :: + TestData2(3, 2) :: Nil, 2).toDF() + df.createOrReplaceTempView("testData2") + df + } + + protected lazy val testData3: DataFrame = { + val df = spark.sparkContext.parallelize( + TestData3(1, None) :: + TestData3(2, Some(2)) :: Nil).toDF() + df.createOrReplaceTempView("testData3") + df + } + + protected lazy val negativeData: DataFrame = { + val df = spark.sparkContext.parallelize( + (1 to 100).map(i => TestData(-i, (-i).toString))).toDF() + df.createOrReplaceTempView("negativeData") + df + } + + protected lazy val largeAndSmallInts: DataFrame = { + val df = spark.sparkContext.parallelize( + LargeAndSmallInts(2147483644, 1) :: + LargeAndSmallInts(1, 2) :: + LargeAndSmallInts(2147483645, 1) :: + LargeAndSmallInts(2, 2) :: + LargeAndSmallInts(2147483646, 1) :: + LargeAndSmallInts(3, 2) :: Nil).toDF() + df.createOrReplaceTempView("largeAndSmallInts") + df + } + + protected lazy val decimalData: DataFrame = { + val df = spark.sparkContext.parallelize( + DecimalData(1, 1) :: + DecimalData(1, 2) :: + DecimalData(2, 1) :: + DecimalData(2, 2) :: + DecimalData(3, 1) :: + DecimalData(3, 2) :: Nil).toDF() + df.createOrReplaceTempView("decimalData") + df + } + + protected lazy val binaryData: DataFrame = { + val df = spark.sparkContext.parallelize( + BinaryData("12".getBytes(StandardCharsets.UTF_8), 1) :: + BinaryData("22".getBytes(StandardCharsets.UTF_8), 5) :: + BinaryData("122".getBytes(StandardCharsets.UTF_8), 3) :: + BinaryData("121".getBytes(StandardCharsets.UTF_8), 2) :: + BinaryData("123".getBytes(StandardCharsets.UTF_8), 4) :: Nil).toDF() + df.createOrReplaceTempView("binaryData") + df + } + + protected lazy val upperCaseData: DataFrame = { + val df = spark.sparkContext.parallelize( + UpperCaseData(1, "A") :: + UpperCaseData(2, "B") :: + UpperCaseData(3, "C") :: + UpperCaseData(4, "D") :: + UpperCaseData(5, "E") :: + UpperCaseData(6, "F") :: Nil).toDF() + df.createOrReplaceTempView("upperCaseData") + df + } + + protected lazy val lowerCaseData: DataFrame = { + val df = spark.sparkContext.parallelize( + LowerCaseData(1, "a") :: + LowerCaseData(2, "b") :: + LowerCaseData(3, "c") :: + LowerCaseData(4, "d") :: Nil).toDF() + df.createOrReplaceTempView("lowerCaseData") + df + } + + protected lazy val arrayData: RDD[ArrayData] = { + val rdd = spark.sparkContext.parallelize( + ArrayData(Seq(1, 2, 3), Seq(Seq(1, 2, 3))) :: + ArrayData(Seq(2, 3, 4), Seq(Seq(2, 3, 4))) :: Nil) + rdd.toDF().createOrReplaceTempView("arrayData") + rdd + } + + protected lazy val mapData: RDD[MapData] = { + val rdd = spark.sparkContext.parallelize( + MapData(Map(1 -> "a1", 2 -> "b1", 3 -> "c1", 4 -> "d1", 5 -> "e1")) :: + MapData(Map(1 -> "a2", 2 -> "b2", 3 -> "c2", 4 -> "d2")) :: + MapData(Map(1 -> "a3", 2 -> "b3", 3 -> "c3")) :: + MapData(Map(1 -> "a4", 2 -> "b4")) :: + MapData(Map(1 -> "a5")) :: Nil) + rdd.toDF().createOrReplaceTempView("mapData") + rdd + } + + protected lazy val repeatedData: RDD[StringData] = { + val rdd = spark.sparkContext.parallelize(List.fill(2)(StringData("test"))) + rdd.toDF().createOrReplaceTempView("repeatedData") + rdd + } + + protected lazy val nullableRepeatedData: RDD[StringData] = { + val rdd = spark.sparkContext.parallelize( + List.fill(2)(StringData(null)) ++ + List.fill(2)(StringData("test"))) + rdd.toDF().createOrReplaceTempView("nullableRepeatedData") + rdd + } + + protected lazy val nullInts: DataFrame = { + val df = spark.sparkContext.parallelize( + NullInts(1) :: + NullInts(2) :: + NullInts(3) :: + NullInts(null) :: Nil).toDF() + df.createOrReplaceTempView("nullInts") + df + } + + protected lazy val allNulls: DataFrame = { + val df = spark.sparkContext.parallelize( + NullInts(null) :: + NullInts(null) :: + NullInts(null) :: + NullInts(null) :: Nil).toDF() + df.createOrReplaceTempView("allNulls") + df + } + + protected lazy val nullStrings: DataFrame = { + val df = spark.sparkContext.parallelize( + NullStrings(1, "abc") :: + NullStrings(2, "ABC") :: + NullStrings(3, null) :: Nil).toDF() + df.createOrReplaceTempView("nullStrings") + df + } + + protected lazy val tableName: DataFrame = { + val df = spark.sparkContext.parallelize(TableName("test") :: Nil).toDF() + df.createOrReplaceTempView("tableName") + df + } + + protected lazy val unparsedStrings: RDD[String] = { + spark.sparkContext.parallelize( + "1, A1, true, null" :: + "2, B2, false, null" :: + "3, C3, true, null" :: + "4, D4, true, 2147483644" :: Nil) + } + + // An RDD with 4 elements and 8 partitions + protected lazy val withEmptyParts: RDD[IntField] = { + val rdd = spark.sparkContext.parallelize((1 to 4).map(IntField), 8) + rdd.toDF().createOrReplaceTempView("withEmptyParts") + rdd + } + + protected lazy val person: DataFrame = { + val df = spark.sparkContext.parallelize( + Person(0, "mike", 30) :: + Person(1, "jim", 20) :: Nil).toDF() + df.createOrReplaceTempView("person") + df + } + + protected lazy val salary: DataFrame = { + val df = spark.sparkContext.parallelize( + Salary(0, 2000.0) :: + Salary(1, 1000.0) :: Nil).toDF() + df.createOrReplaceTempView("salary") + df + } + + protected lazy val complexData: DataFrame = { + val df = spark.sparkContext.parallelize( + ComplexData(Map("1" -> 1), TestData(1, "1"), Seq(1, 1, 1), true) :: + ComplexData(Map("2" -> 2), TestData(2, "2"), Seq(2, 2, 2), false) :: + Nil).toDF() + df.createOrReplaceTempView("complexData") + df + } + + protected lazy val courseSales: DataFrame = { + val df = spark.sparkContext.parallelize( + CourseSales("dotNET", 2012, 10000) :: + CourseSales("Java", 2012, 20000) :: + CourseSales("dotNET", 2012, 5000) :: + CourseSales("dotNET", 2013, 48000) :: + CourseSales("Java", 2013, 30000) :: Nil).toDF() + df.createOrReplaceTempView("courseSales") + df + } + + /** + * Initialize all test data such that all temp tables are properly registered. + */ + def loadTestData(): Unit = { + assert(spark != null, "attempted to initialize test data before SparkSession.") + emptyTestData + testData + testData2 + testData3 + negativeData + largeAndSmallInts + decimalData + binaryData + upperCaseData + lowerCaseData + arrayData + mapData + repeatedData + nullableRepeatedData + nullInts + allNulls + nullStrings + tableName + unparsedStrings + withEmptyParts + person + salary + complexData + courseSales + } +} + +/** + * Case classes used in test data. + */ +private[sql] object SQLTestData { + case class TestData(key: Int, value: String) + case class TestData2(a: Int, b: Int) + case class TestData3(a: Int, b: Option[Int]) + case class LargeAndSmallInts(a: Int, b: Int) + case class DecimalData(a: BigDecimal, b: BigDecimal) + case class BinaryData(a: Array[Byte], b: Int) + case class UpperCaseData(N: Int, L: String) + case class LowerCaseData(n: Int, l: String) + case class ArrayData(data: Seq[Int], nestedData: Seq[Seq[Int]]) + case class MapData(data: scala.collection.Map[Int, String]) + case class StringData(s: String) + case class IntField(i: Int) + case class NullInts(a: Integer) + case class NullStrings(n: Int, s: String) + case class TableName(tableName: String) + case class Person(id: Int, name: String, age: Int) + case class Salary(personId: Int, salary: Double) + case class ComplexData(m: Map[String, Int], s: TestData, a: Seq[Int], b: Boolean) + case class CourseSales(course: String, year: Int, earnings: Double) +} diff --git a/spark/spark-2.2/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/spark/spark-2.2/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala new file mode 100644 index 000000000..1e48e7116 --- /dev/null +++ b/spark/spark-2.2/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -0,0 +1,336 @@ +/* + * 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.util.UUID + +import scala.language.implicitConversions +import scala.util.Try +import scala.util.control.NonFatal + +import org.apache.hadoop.conf.Configuration +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException +import org.apache.spark.sql.catalyst.catalog.SessionCatalog.DEFAULT_DATABASE +import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.execution.FilterExec +import org.apache.spark.util.{UninterruptibleThread, Utils} + +/** + * Helper trait that should be extended by all SQL test suites. + * + * This allows subclasses to plugin a custom [[SQLContext]]. It comes with test data + * prepared in advance as well as all implicit conversions used extensively by dataframes. + * To use implicit methods, import `testImplicits._` instead of through the [[SQLContext]]. + * + * Subclasses should *not* create [[SQLContext]]s in the test suite constructor, which is + * prone to leaving multiple overlapping [[org.apache.spark.SparkContext]]s in the same JVM. + */ +private[sql] trait SQLTestUtils + extends SparkFunSuite + with BeforeAndAfterAll + with SQLTestData { self => + + protected def sparkContext = spark.sparkContext + + // Whether to materialize all test data before the first test is run + private var loadTestDataBeforeTests = false + + // Shorthand for running a query using our SQLContext + protected lazy val sql = spark.sql _ + + /** + * A helper object for importing SQL implicits. + * + * Note that the alternative of importing `spark.implicits._` is not possible here. + * This is because we create the [[SQLContext]] immediately before the first test is run, + * but the implicits import is needed in the constructor. + */ + protected object testImplicits extends SQLImplicits { + protected override def _sqlContext: SQLContext = self.spark.sqlContext + } + + /** + * Materialize the test data immediately after the [[SQLContext]] is set up. + * This is necessary if the data is accessed by name but not through direct reference. + */ + protected def setupTestData(): Unit = { + loadTestDataBeforeTests = true + } + + protected override def beforeAll(): Unit = { + super.beforeAll() + if (loadTestDataBeforeTests) { + loadTestData() + } + } + + /** + * Sets all SQL configurations specified in `pairs`, calls `f`, and then restore all SQL + * configurations. + * + * @todo Probably this method should be moved to a more general place + */ + protected def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { + val (keys, values) = pairs.unzip + val currentValues = keys.map(key => Try(spark.conf.get(key)).toOption) + (keys, values).zipped.foreach(spark.conf.set) + try f finally { + keys.zip(currentValues).foreach { + case (key, Some(value)) => spark.conf.set(key, value) + case (key, None) => spark.conf.unset(key) + } + } + } + + /** + * Generates a temporary path without creating the actual file/directory, then pass it to `f`. If + * a file/directory is created there by `f`, it will be delete after `f` returns. + * + * @todo Probably this method should be moved to a more general place + */ + protected def withTempPath(f: File => Unit): Unit = { + val path = Utils.createTempDir() + path.delete() + try f(path) finally Utils.deleteRecursively(path) + } + + /** + * Creates a temporary directory, which is then passed to `f` and will be deleted after `f` + * returns. + * + * @todo Probably this method should be moved to a more general place + */ + protected def withTempDir(f: File => Unit): Unit = { + val dir = Utils.createTempDir().getCanonicalFile + try f(dir) finally Utils.deleteRecursively(dir) + } + + /** + * Drops functions after calling `f`. A function is represented by (functionName, isTemporary). + */ + protected def withUserDefinedFunction(functions: (String, Boolean)*)(f: => Unit): Unit = { + try { + f + } catch { + case cause: Throwable => throw cause + } finally { + // If the test failed part way, we don't want to mask the failure by failing to remove + // temp tables that never got created. + functions.foreach { case (functionName, isTemporary) => + val withTemporary = if (isTemporary) "TEMPORARY" else "" + spark.sql(s"DROP $withTemporary FUNCTION IF EXISTS $functionName") + assert( + !spark.sessionState.catalog.functionExists(FunctionIdentifier(functionName)), + s"Function $functionName should have been dropped. But, it still exists.") + } + } + } + + /** + * Drops temporary table `tableName` after calling `f`. + */ + protected def withTempView(tableNames: String*)(f: => Unit): Unit = { + try f finally { + // If the test failed part way, we don't want to mask the failure by failing to remove + // temp tables that never got created. + try tableNames.foreach(spark.catalog.dropTempView) catch { + case _: NoSuchTableException => + } + } + } + + /** + * Drops table `tableName` after calling `f`. + */ + protected def withTable(tableNames: String*)(f: => Unit): Unit = { + try f finally { + tableNames.foreach { name => + spark.sql(s"DROP TABLE IF EXISTS $name") + } + } + } + + /** + * Drops view `viewName` after calling `f`. + */ + protected def withView(viewNames: String*)(f: => Unit): Unit = { + try f finally { + viewNames.foreach { name => + spark.sql(s"DROP VIEW IF EXISTS $name") + } + } + } + + /** + * Creates a temporary database and switches current database to it before executing `f`. This + * database is dropped after `f` returns. + * + * Note that this method doesn't switch current database before executing `f`. + */ + protected def withTempDatabase(f: String => Unit): Unit = { + val dbName = s"db_${UUID.randomUUID().toString.replace('-', '_')}" + + try { + spark.sql(s"CREATE DATABASE $dbName") + } catch { case cause: Throwable => + fail("Failed to create temporary database", cause) + } + + try f(dbName) finally { + if (spark.catalog.currentDatabase == dbName) { + spark.sql(s"USE ${DEFAULT_DATABASE}") + } + spark.sql(s"DROP DATABASE $dbName CASCADE") + } + } + + /** + * Activates database `db` before executing `f`, then switches back to `default` database after + * `f` returns. + */ + protected def activateDatabase(db: String)(f: => Unit): Unit = { + spark.sessionState.catalog.setCurrentDatabase(db) + try f finally spark.sessionState.catalog.setCurrentDatabase("default") + } + + /** + * Strip Spark-side filtering in order to check if a datasource filters rows correctly. + */ + protected def stripSparkFilter(df: DataFrame): DataFrame = { + val schema = df.schema + val withoutFilters = df.queryExecution.sparkPlan.transform { + case FilterExec(_, child) => child + } + + spark.internalCreateDataFrame(withoutFilters.execute(), schema) + } + + /** + * Turn a logical plan into a [[DataFrame]]. This should be removed once we have an easier + * way to construct [[DataFrame]] directly out of local data without relying on implicits. + */ + protected implicit def logicalPlanToSparkQuery(plan: LogicalPlan): DataFrame = { + Dataset.ofRows(spark, plan) + } + + /** + * Disable stdout and stderr when running the test. To not output the logs to the console, + * ConsoleAppender's `follow` should be set to `true` so that it will honors reassignments of + * System.out or System.err. Otherwise, ConsoleAppender will still output to the console even if + * we change System.out and System.err. + */ + protected def testQuietly(name: String)(f: => Unit): Unit = { + test(name) { + quietly { + f + } + } + } + + /** Run a test on a separate [[UninterruptibleThread]]. */ + protected def testWithUninterruptibleThread(name: String, quietly: Boolean = false) + (body: => Unit): Unit = { + val timeoutMillis = 10000 + @transient var ex: Throwable = null + + def runOnThread(): Unit = { + val thread = new UninterruptibleThread(s"Testing thread for test $name") { + override def run(): Unit = { + try { + body + } catch { + case NonFatal(e) => + ex = e + } + } + } + thread.setDaemon(true) + thread.start() + thread.join(timeoutMillis) + if (thread.isAlive) { + thread.interrupt() + // If this interrupt does not work, then this thread is most likely running something that + // is not interruptible. There is not much point to wait for the thread to termniate, and + // we rather let the JVM terminate the thread on exit. + fail( + s"Test '$name' running on o.a.s.util.UninterruptibleThread timed out after" + + s" $timeoutMillis ms") + } else if (ex != null) { + throw ex + } + } + + if (quietly) { + testQuietly(name) { runOnThread() } + } else { + test(name) { runOnThread() } + } + } +} + +private[sql] object SQLTestUtils { + + def compareAnswers( + sparkAnswer: Seq[Row], + expectedAnswer: Seq[Row], + sort: Boolean): Option[String] = { + def prepareAnswer(answer: Seq[Row]): Seq[Row] = { + // Converts data to types that we can do equality comparison using Scala collections. + // For BigDecimal type, the Scala type has a better definition of equality test (similar to + // Java's java.math.BigDecimal.compareTo). + // For binary arrays, we convert it to Seq to avoid of calling java.util.Arrays.equals for + // equality test. + // This function is copied from Catalyst's QueryTest + val converted: Seq[Row] = answer.map { s => + Row.fromSeq(s.toSeq.map { + case d: java.math.BigDecimal => BigDecimal(d) + case b: Array[Byte] => b.toSeq + case o => o + }) + } + if (sort) { + converted.sortBy(_.toString()) + } else { + converted + } + } + if (prepareAnswer(expectedAnswer) != prepareAnswer(sparkAnswer)) { + val errorMessage = + s""" + | == Results == + | ${sideBySide( + s"== Expected Answer - ${expectedAnswer.size} ==" +: + prepareAnswer(expectedAnswer).map(_.toString()), + s"== Actual Answer - ${sparkAnswer.size} ==" +: + prepareAnswer(sparkAnswer).map(_.toString())).mkString("\n")} + """.stripMargin + Some(errorMessage) + } else { + None + } + } +} diff --git a/spark/spark-2.2/src/test/scala/org/apache/spark/sql/test/VectorQueryTest.scala b/spark/spark-2.2/src/test/scala/org/apache/spark/sql/test/VectorQueryTest.scala new file mode 100644 index 000000000..4e2a0c188 --- /dev/null +++ b/spark/spark-2.2/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) + } + } +} diff --git a/spark/spark-2.2/src/test/scala/org/apache/spark/streaming/HivemallOpsWithFeatureSuite.scala b/spark/spark-2.2/src/test/scala/org/apache/spark/streaming/HivemallOpsWithFeatureSuite.scala new file mode 100644 index 000000000..0e1372dcc --- /dev/null +++ b/spark/spark-2.2/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.(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.(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(_ => {}) + } + } +} diff --git a/spark/spark-2.2/src/test/scala/org/apache/spark/test/TestUtils.scala b/spark/spark-2.2/src/test/scala/org/apache/spark/test/TestUtils.scala new file mode 100644 index 000000000..fa7b6e54e --- /dev/null +++ b/spark/spark-2.2/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) + } +} From cbda47a8fcd667028256c722c0905d0553ea7945 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Thu, 27 Jul 2017 23:54:13 +0900 Subject: [PATCH 2/5] Add enforce-plugin to validate java source/target versions --- pom.xml | 33 +++++++++++++++++++++++++++++++++ 1 file changed, 33 insertions(+) diff --git a/pom.xml b/pom.xml index 453209176..8a543e64d 100644 --- a/pom.xml +++ b/pom.xml @@ -276,6 +276,39 @@ 2.2.0 2.2 + + + + + org.apache.maven.plugins + maven-enforcer-plugin + 1.4.1 + + + enforce-versions + validate + + enforce + + + + + java.source.version + 1.8 + When -Pspark-2.2 set, java.source.version must be 1.8 + + + java.target.version + 1.8 + When -Pspark-2.2 set, java.target.version must be 1.8 + + + + + + + + spark-2.1 From 18df884a0d36a7cd272f712c2b4414b212d958ee Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Fri, 28 Jul 2017 00:22:14 +0900 Subject: [PATCH 3/5] Fix style errors --- .../spark/sql/catalyst/plans/PlanTest.scala | 26 ++++++++++--------- 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/spark/spark-2.2/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/spark/spark-2.2/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index f44428c35..a4aeaa63f 100644 --- a/spark/spark-2.2/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/spark/spark-2.2/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -1,18 +1,20 @@ /* - * 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 + * 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 + * 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. + * 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.catalyst.plans From 55fda48afbc1af3f95ea5b40d0645b4d149cba72 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Fri, 28 Jul 2017 00:22:24 +0900 Subject: [PATCH 4/5] Update .travis.yml --- .travis.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.travis.yml b/.travis.yml index 96f8f4e81..75d8df85a 100644 --- a/.travis.yml +++ b/.travis.yml @@ -36,6 +36,7 @@ notifications: script: - mvn -q scalastyle:check test -Pspark-2.1 # test the spark-2.0 modules only in the following runs + - if [[ ! -z "$(java -version 2>&1 | grep 1.8)" ]]; then mvn -q scalastyle:check clean -Pspark-2.2 -Djava.source.version=1.8 -Djava.target.version=1.8 -pl spark/spark-2.2 -am test -Dtest=none; fi - mvn -q scalastyle:check clean -Pspark-2.0 -pl spark/spark-2.0 -am test -Dtest=none after_success: From 95ec7833032701d6e87b19cad0ebedbc0a8f6cf4 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Fri, 28 Jul 2017 11:09:21 +0900 Subject: [PATCH 5/5] Add bin/run_travis_tests.sh --- .travis.yml | 5 +---- bin/run_travis_tests.sh | 47 +++++++++++++++++++++++++++++++++++++++++ 2 files changed, 48 insertions(+), 4 deletions(-) create mode 100755 bin/run_travis_tests.sh diff --git a/.travis.yml b/.travis.yml index 75d8df85a..c64c5ffa3 100644 --- a/.travis.yml +++ b/.travis.yml @@ -34,10 +34,7 @@ notifications: email: false script: - - mvn -q scalastyle:check test -Pspark-2.1 - # test the spark-2.0 modules only in the following runs - - if [[ ! -z "$(java -version 2>&1 | grep 1.8)" ]]; then mvn -q scalastyle:check clean -Pspark-2.2 -Djava.source.version=1.8 -Djava.target.version=1.8 -pl spark/spark-2.2 -am test -Dtest=none; fi - - mvn -q scalastyle:check clean -Pspark-2.0 -pl spark/spark-2.0 -am test -Dtest=none + - ./bin/run_travis_tests.sh after_success: - mvn clean cobertura:cobertura coveralls:report diff --git a/bin/run_travis_tests.sh b/bin/run_travis_tests.sh new file mode 100755 index 000000000..f1bffec57 --- /dev/null +++ b/bin/run_travis_tests.sh @@ -0,0 +1,47 @@ +#!/bin/sh +# +# 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. +# + +if [ "$HIVEMALL_HOME" = "" ]; then + if [ -e ../bin/${0##*/} ]; then + HIVEMALL_HOME=".." + elif [ -e ./bin/${0##*/} ]; then + HIVEMALL_HOME="." + else + echo "env HIVEMALL_HOME not defined" + exit 1 + fi +fi + +set -ev + +cd $HIVEMALL_HOME + +mvn -q scalastyle:check test -Pspark-2.1 + +# Tests the spark-2.2/spark-2.0 modules only in the following runs +if [[ ! -z "$(java -version 2>&1 | grep 1.8)" ]]; then + mvn -q scalastyle:check clean -Djava.source.version=1.8 -Djava.target.version=1.8 \ + -Pspark-2.2 -pl spark/spark-2.2 -am test -Dtest=none +fi + +mvn -q scalastyle:check clean -Pspark-2.0 -pl spark/spark-2.0 -am test -Dtest=none + +exit 0 +