From fc148ce9ac30af4bee3a8319c192bc4b01816706 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Thu, 29 Mar 2018 08:57:38 +0900 Subject: [PATCH] Drop the Spark-2.0 support --- bin/run_travis_tests.sh | 2 +- spark/pom.xml | 1 - spark/spark-2.0/bin/mvn-zinc | 99 -- spark/spark-2.0/extra-src/README | 1 - .../org/apache/spark/sql/hive/HiveShim.scala | 280 ---- spark/spark-2.0/pom.xml | 138 -- .../hivemall/xgboost/XGBoostOptions.scala | 58 - ...pache.spark.sql.sources.DataSourceRegister | 1 - .../src/main/resources/log4j.properties | 29 - .../hivemall/tools/RegressionDatagen.scala | 66 - .../sql/catalyst/expressions/EachTopK.scala | 116 -- .../sql/hive/HivemallGroupedDataset.scala | 303 ---- .../apache/spark/sql/hive/HivemallOps.scala | 1381 ----------------- .../apache/spark/sql/hive/HivemallUtils.scala | 145 -- .../sql/hive/internal/HivemallOpsImpl.scala | 78 - .../sql/hive/source/XGBoostFileFormat.scala | 146 -- .../streaming/HivemallStreamingOps.scala | 47 - .../src/test/resources/data/files/README.md | 22 - .../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 | 24 - .../hivemall/mix/server/MixServerSuite.scala | 125 -- .../tools/RegressionDatagenSuite.scala | 32 - .../org/apache/spark/SparkFunSuite.scala | 50 - .../feature/HivemallLabeledPointSuite.scala | 35 - .../org/apache/spark/sql/QueryTest.scala | 475 ------ .../spark/sql/catalyst/plans/PlanTest.scala | 61 - .../apache/spark/sql/hive/HiveUdfSuite.scala | 160 -- .../spark/sql/hive/HivemallOpsSuite.scala | 811 ---------- .../spark/sql/hive/ModelMixingSuite.scala | 285 ---- .../apache/spark/sql/hive/XGBoostSuite.scala | 143 -- .../sql/hive/benchmark/MiscBenchmark.scala | 224 --- .../hive/test/HivemallFeatureQueryTest.scala | 112 -- .../sql/hive/test/TestHiveSingleton.scala | 37 - .../HivemallOpsWithFeatureSuite.scala | 154 -- .../org/apache/spark/test/TestUtils.scala | 64 - .../apache/spark/test/VectorQueryTest.scala | 89 -- 40 files changed, 1 insertion(+), 5793 deletions(-) delete mode 100755 spark/spark-2.0/bin/mvn-zinc delete mode 100644 spark/spark-2.0/extra-src/README delete mode 100644 spark/spark-2.0/extra-src/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala delete mode 100644 spark/spark-2.0/pom.xml delete mode 100644 spark/spark-2.0/src/main/java/hivemall/xgboost/XGBoostOptions.scala delete mode 100644 spark/spark-2.0/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister delete mode 100644 spark/spark-2.0/src/main/resources/log4j.properties delete mode 100644 spark/spark-2.0/src/main/scala/hivemall/tools/RegressionDatagen.scala delete mode 100644 spark/spark-2.0/src/main/scala/org/apache/spark/sql/catalyst/expressions/EachTopK.scala delete mode 100644 spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallGroupedDataset.scala delete mode 100644 spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallOps.scala delete mode 100644 spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallUtils.scala delete mode 100644 spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/internal/HivemallOpsImpl.scala delete mode 100644 spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/source/XGBoostFileFormat.scala delete mode 100644 spark/spark-2.0/src/main/scala/org/apache/spark/streaming/HivemallStreamingOps.scala delete mode 100644 spark/spark-2.0/src/test/resources/data/files/README.md delete mode 100644 spark/spark-2.0/src/test/resources/data/files/complex.seq delete mode 100644 spark/spark-2.0/src/test/resources/data/files/episodes.avro delete mode 100644 spark/spark-2.0/src/test/resources/data/files/json.txt delete mode 100644 spark/spark-2.0/src/test/resources/data/files/kv1.txt delete mode 100644 spark/spark-2.0/src/test/resources/data/files/kv3.txt delete mode 100644 spark/spark-2.0/src/test/resources/log4j.properties delete mode 100644 spark/spark-2.0/src/test/scala/hivemall/mix/server/MixServerSuite.scala delete mode 100644 spark/spark-2.0/src/test/scala/hivemall/tools/RegressionDatagenSuite.scala delete mode 100644 spark/spark-2.0/src/test/scala/org/apache/spark/SparkFunSuite.scala delete mode 100644 spark/spark-2.0/src/test/scala/org/apache/spark/ml/feature/HivemallLabeledPointSuite.scala delete mode 100644 spark/spark-2.0/src/test/scala/org/apache/spark/sql/QueryTest.scala delete mode 100644 spark/spark-2.0/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala delete mode 100644 spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/HiveUdfSuite.scala delete mode 100644 spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/HivemallOpsSuite.scala delete mode 100644 spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/ModelMixingSuite.scala delete mode 100644 spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/XGBoostSuite.scala delete mode 100644 spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/benchmark/MiscBenchmark.scala delete mode 100644 spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/test/HivemallFeatureQueryTest.scala delete mode 100644 spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala delete mode 100644 spark/spark-2.0/src/test/scala/org/apache/spark/streaming/HivemallOpsWithFeatureSuite.scala delete mode 100644 spark/spark-2.0/src/test/scala/org/apache/spark/test/TestUtils.scala delete mode 100644 spark/spark-2.0/src/test/scala/org/apache/spark/test/VectorQueryTest.scala diff --git a/bin/run_travis_tests.sh b/bin/run_travis_tests.sh index 9693520ab..b9e0763b9 100755 --- a/bin/run_travis_tests.sh +++ b/bin/run_travis_tests.sh @@ -35,7 +35,7 @@ cd $HIVEMALL_HOME/spark export MAVEN_OPTS="-XX:MaxPermSize=256m" -mvn -q scalastyle:check -pl spark-2.0,spark-2.1 -am test +mvn -q scalastyle:check -pl spark-2.1 -am test # spark-2.2 runs on Java 8+ if [[ ! -z "$(java -version 2>&1 | grep 1.8)" ]]; then diff --git a/spark/pom.xml b/spark/pom.xml index 27bb6dbe8..99553f954 100644 --- a/spark/pom.xml +++ b/spark/pom.xml @@ -32,7 +32,6 @@ common - spark-2.0 spark-2.1 spark-2.2 spark-2.3 diff --git a/spark/spark-2.0/bin/mvn-zinc b/spark/spark-2.0/bin/mvn-zinc deleted file mode 100755 index 759b0a56d..000000000 --- a/spark/spark-2.0/bin/mvn-zinc +++ /dev/null @@ -1,99 +0,0 @@ -#!/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.0/extra-src/README b/spark/spark-2.0/extra-src/README deleted file mode 100644 index 8b5d0cd51..000000000 --- a/spark/spark-2.0/extra-src/README +++ /dev/null @@ -1 +0,0 @@ -Copyed from spark master [commit 908e37bcc10132bb2aa7f80ae694a9df6e40f31a] diff --git a/spark/spark-2.0/extra-src/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala b/spark/spark-2.0/extra-src/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala deleted file mode 100644 index a69584119..000000000 --- a/spark/spark-2.0/extra-src/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala +++ /dev/null @@ -1,280 +0,0 @@ -/* - * 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 or empty - */ - def appendReadColumns(conf: Configuration, ids: Seq[Integer], names: Seq[String]) { - if (ids != null && ids.nonEmpty) { - ColumnProjectionUtils.appendReadColumns(conf, ids.asJava) - } - if (names != null && names.nonEmpty) { - 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.0/pom.xml b/spark/spark-2.0/pom.xml deleted file mode 100644 index 54c817d1d..000000000 --- a/spark/spark-2.0/pom.xml +++ /dev/null @@ -1,138 +0,0 @@ - - - 4.0.0 - - - org.apache.hivemall - hivemall-spark - 0.5.1-incubating-SNAPSHOT - ../pom.xml - - - hivemall-spark2.0 - Hivemall on Spark 2.0 - jar - - - ${project.parent.parent.basedir} - 2.0.2 - 2.0 - - - - - - org.apache.hivemall - hivemall-core - compile - - - org.apache.hivemall - hivemall-xgboost - compile - - - org.apache.hivemall - hivemall-spark-common - ${project.version} - compile - - - - - org.scala-lang - scala-library - provided - - - 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 - - - - - org.apache.hivemall - hivemall-mixserv - test - - - org.scalatest - scalatest_${scala.binary.version} - test - - - - - - - - org.apache.maven.plugins - maven-shade-plugin - - - - org.apache.maven.plugins - maven-surefire-plugin - - true - - - - - org.scalatest - scalatest-maven-plugin - - - test - - test - - - - - - - diff --git a/spark/spark-2.0/src/main/java/hivemall/xgboost/XGBoostOptions.scala b/spark/spark-2.0/src/main/java/hivemall/xgboost/XGBoostOptions.scala deleted file mode 100644 index 48a773b6d..000000000 --- a/spark/spark-2.0/src/main/java/hivemall/xgboost/XGBoostOptions.scala +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.0/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/spark/spark-2.0/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister deleted file mode 100644 index b49e20a2a..000000000 --- a/spark/spark-2.0/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ /dev/null @@ -1 +0,0 @@ -org.apache.spark.sql.hive.source.XGBoostFileFormat diff --git a/spark/spark-2.0/src/main/resources/log4j.properties b/spark/spark-2.0/src/main/resources/log4j.properties deleted file mode 100644 index ef4f6063b..000000000 --- a/spark/spark-2.0/src/main/resources/log4j.properties +++ /dev/null @@ -1,29 +0,0 @@ -# -# 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. -# - -# 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.0/src/main/scala/hivemall/tools/RegressionDatagen.scala b/spark/spark-2.0/src/main/scala/hivemall/tools/RegressionDatagen.scala deleted file mode 100644 index 72a5c832c..000000000 --- a/spark/spark-2.0/src/main/scala/hivemall/tools/RegressionDatagen.scala +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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.0/src/main/scala/org/apache/spark/sql/catalyst/expressions/EachTopK.scala b/spark/spark-2.0/src/main/scala/org/apache/spark/sql/catalyst/expressions/EachTopK.scala deleted file mode 100644 index f1312ba2f..000000000 --- a/spark/spark-2.0/src/main/scala/org/apache/spark/sql/catalyst/expressions/EachTopK.scala +++ /dev/null @@ -1,116 +0,0 @@ -/* - * 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.CodegenFallback -import org.apache.spark.sql.catalyst.util.TypeUtils -import org.apache.spark.sql.types._ -import org.apache.spark.util.BoundedPriorityQueue - -case class EachTopK( - k: Int, - groupingExpression: Expression, - scoreExpression: Expression, - children: Seq[Attribute]) extends Generator with CodegenFallback { - type QueueType = (AnyRef, InternalRow) - - require(k != 0, "`k` must not have 0") - - private[this] lazy val scoreType = scoreExpression.dataType - private[this] lazy val scoreOrdering = { - val ordering = TypeUtils.getInterpretedOrdering(scoreType) - .asInstanceOf[Ordering[AnyRef]] - if (k > 0) { - ordering - } else { - ordering.reverse - } - } - private[this] lazy val reverseScoreOrdering = scoreOrdering.reverse - - private[this] val queue: BoundedPriorityQueue[QueueType] = { - new BoundedPriorityQueue(Math.abs(k))(new Ordering[QueueType] { - override def compare(x: QueueType, y: QueueType): Int = - scoreOrdering.compare(x._1, y._1) - }) - } - - lazy private[this] val groupingProjection: UnsafeProjection = - UnsafeProjection.create(groupingExpression :: Nil, children) - - lazy private[this] val scoreProjection: UnsafeProjection = - UnsafeProjection.create(scoreExpression :: Nil, children) - - // The grouping key of the current partition - private[this] var currentGroupingKey: UnsafeRow = _ - - override def checkInputDataTypes(): TypeCheckResult = { - if (!TypeCollection.Ordered.acceptsType(scoreExpression.dataType)) { - TypeCheckResult.TypeCheckFailure( - s"$scoreExpression must have a comparable type") - } else { - TypeCheckResult.TypeCheckSuccess - } - } - - override def elementSchema: StructType = - StructType( - Seq(StructField("rank", IntegerType)) ++ - children.map(d => StructField(d.prettyName, d.dataType)) - ) - - 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) - } - outputRows.zip(rankNum.map(_._1)).map { case ((_, row), index) => - new JoinedRow(InternalRow(index), row) - } - } else { - Seq.empty - } - - override def eval(input: InternalRow): TraversableOnce[InternalRow] = { - val groupingKey = groupingProjection(input) - val ret = if (currentGroupingKey != groupingKey) { - val topKRows = topKRowsForGroup() - currentGroupingKey = groupingKey.copy() - queue.clear() - topKRows - } else { - Iterator.empty - } - queue += Tuple2(scoreProjection(input).get(0, scoreType), input.copy()) - ret - } - - override def terminate(): TraversableOnce[InternalRow] = { - if (queue.size > 0) { - val topKRows = topKRowsForGroup() - queue.clear() - topKRows - } else { - Iterator.empty - } - } -} diff --git a/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallGroupedDataset.scala b/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallGroupedDataset.scala deleted file mode 100644 index 7d2cd8366..000000000 --- a/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallGroupedDataset.scala +++ /dev/null @@ -1,303 +0,0 @@ -/* - * 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.F1ScoreUDAF - * @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.F1ScoreUDAF"), - 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.0/src/main/scala/org/apache/spark/sql/hive/HivemallOps.scala b/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallOps.scala deleted file mode 100644 index b5299efe3..000000000 --- a/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallOps.scala +++ /dev/null @@ -1,1381 +0,0 @@ -/* - * 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.{EachTopK, Expression, Literal, NamedExpression, UserDefinedGenerator} -import org.apache.spark.sql.catalyst.plans.logical.{Generate, LogicalPlan, Project} -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] val _sparkSession = df.sparkSession - private[this] val _analyzer = _sparkSession.sessionState.analyzer - - /** - * @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 { - val outputAttr = exprs.drop(1).map { - case Column(expr: NamedExpression) => UnresolvedAttribute(expr.name) - case Column(expr: Expression) => UnresolvedAttribute(expr.simpleString) - } - 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, group: Column, score: 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) - } - val clusterDf = df.repartition(group).sortWithinPartitions(group) - val child = clusterDf.logicalPlan - val logicalPlan = Project(group.named +: score.named +: child.output, child) - _analyzer.execute(logicalPlan) match { - case Project(group :: score :: origCols, c) => - Generate( - EachTopK(kInt, group, score, c.output), - join = false, outer = false, None, - (Seq("rank") ++ origCols.map(_.name)).map(UnresolvedAttribute(_)), - clusterDf.logicalPlan - ) - } - } - - /** - * @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 = df.sparkSession.sessionState.executePlan(logicalPlan) - val outputSchema = queryExecution.sparkPlan.schema - new Dataset[Row](df.sparkSession, queryExecution, RowEncoder(outputSchema)) - } -} - -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") - - /** - * 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.0/src/main/scala/org/apache/spark/sql/hive/HivemallUtils.scala b/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallUtils.scala deleted file mode 100644 index 056d6d62a..000000000 --- a/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/HivemallUtils.scala +++ /dev/null @@ -1,145 +0,0 @@ -/* - * 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.0/src/main/scala/org/apache/spark/sql/hive/internal/HivemallOpsImpl.scala b/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/internal/HivemallOpsImpl.scala deleted file mode 100644 index ab5c5fbf9..000000000 --- a/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/internal/HivemallOpsImpl.scala +++ /dev/null @@ -1,78 +0,0 @@ -/* - * 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.0/src/main/scala/org/apache/spark/sql/hive/source/XGBoostFileFormat.scala b/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/source/XGBoostFileFormat.scala deleted file mode 100644 index 9cdc09f01..000000000 --- a/spark/spark-2.0/src/main/scala/org/apache/spark/sql/hive/source/XGBoostFileFormat.scala +++ /dev/null @@ -1,146 +0,0 @@ -/* - * 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.mapreduce._ - -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: Row): Unit = { - val modelId = row.getString(0) - val model = row.get(1).asInstanceOf[Array[Byte]] - val filePath = new Path(new URI(s"$path/$modelId")) - val fs = filePath.getFileSystem(hadoopConf.value) - val outputFile = fs.create(filePath) - outputFile.write(model) - outputFile.close() - } - - override def close(): Unit = {} -} - -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, - bucketId: Option[Int], - dataSchema: StructType, - context: TaskAttemptContext): OutputWriter = { - if (bucketId.isDefined) { - sys.error("XGBoostFileFormat doesn't support bucketing") - } - new XGBoostOutputWriter(path, dataSchema, context) - } - } - } - - 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.0/src/main/scala/org/apache/spark/streaming/HivemallStreamingOps.scala b/spark/spark-2.0/src/main/scala/org/apache/spark/streaming/HivemallStreamingOps.scala deleted file mode 100644 index a6bbb4b57..000000000 --- a/spark/spark-2.0/src/main/scala/org/apache/spark/streaming/HivemallStreamingOps.scala +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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.{DataFrame, Row, SQLContext} -import org.apache.spark.streaming.dstream.DStream - -final class HivemallStreamingOps(ds: DStream[HivemallLabeledPoint]) { - - def predict[U: ClassTag](f: DataFrame => DataFrame)(implicit sqlContext: SQLContext) - : DStream[Row] = { - ds.transform[Row] { rdd: RDD[HivemallLabeledPoint] => - f(sqlContext.createDataFrame(rdd)).rdd - } - } -} - -object HivemallStreamingOps { - - /** - * Implicitly inject the [[HivemallStreamingOps]] into [[DStream]]. - */ - implicit def dataFrameToHivemallStreamingOps(ds: DStream[HivemallLabeledPoint]) - : HivemallStreamingOps = { - new HivemallStreamingOps(ds) - } -} diff --git a/spark/spark-2.0/src/test/resources/data/files/README.md b/spark/spark-2.0/src/test/resources/data/files/README.md deleted file mode 100644 index 238d4721f..000000000 --- a/spark/spark-2.0/src/test/resources/data/files/README.md +++ /dev/null @@ -1,22 +0,0 @@ - - -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.0/src/test/resources/data/files/complex.seq b/spark/spark-2.0/src/test/resources/data/files/complex.seq deleted file mode 100644 index e69de29bb..000000000 diff --git a/spark/spark-2.0/src/test/resources/data/files/episodes.avro b/spark/spark-2.0/src/test/resources/data/files/episodes.avro deleted file mode 100644 index e69de29bb..000000000 diff --git a/spark/spark-2.0/src/test/resources/data/files/json.txt b/spark/spark-2.0/src/test/resources/data/files/json.txt deleted file mode 100644 index e69de29bb..000000000 diff --git a/spark/spark-2.0/src/test/resources/data/files/kv1.txt b/spark/spark-2.0/src/test/resources/data/files/kv1.txt deleted file mode 100644 index e69de29bb..000000000 diff --git a/spark/spark-2.0/src/test/resources/data/files/kv3.txt b/spark/spark-2.0/src/test/resources/data/files/kv3.txt deleted file mode 100644 index e69de29bb..000000000 diff --git a/spark/spark-2.0/src/test/resources/log4j.properties b/spark/spark-2.0/src/test/resources/log4j.properties deleted file mode 100644 index c6e4297e1..000000000 --- a/spark/spark-2.0/src/test/resources/log4j.properties +++ /dev/null @@ -1,24 +0,0 @@ -# -# 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. -# - -# 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.0/src/test/scala/hivemall/mix/server/MixServerSuite.scala b/spark/spark-2.0/src/test/scala/hivemall/mix/server/MixServerSuite.scala deleted file mode 100644 index cadc852d0..000000000 --- a/spark/spark-2.0/src/test/scala/hivemall/mix/server/MixServerSuite.scala +++ /dev/null @@ -1,125 +0,0 @@ -/* - * 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.client.MixClient -import hivemall.mix.MixMessage.MixEventName -import hivemall.mix.server.MixServer.ServerState -import hivemall.model.{DenseModel, PredictionModel} -import hivemall.model.{NewDenseModel, PredictionModel} -import hivemall.model.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 NewDenseModel(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.0/src/test/scala/hivemall/tools/RegressionDatagenSuite.scala b/spark/spark-2.0/src/test/scala/hivemall/tools/RegressionDatagenSuite.scala deleted file mode 100644 index 8c068376f..000000000 --- a/spark/spark-2.0/src/test/scala/hivemall/tools/RegressionDatagenSuite.scala +++ /dev/null @@ -1,32 +0,0 @@ -/* - * 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.0/src/test/scala/org/apache/spark/SparkFunSuite.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/SparkFunSuite.scala deleted file mode 100644 index 0b101c828..000000000 --- a/spark/spark-2.0/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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.0/src/test/scala/org/apache/spark/ml/feature/HivemallLabeledPointSuite.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/ml/feature/HivemallLabeledPointSuite.scala deleted file mode 100644 index f57983fc9..000000000 --- a/spark/spark-2.0/src/test/scala/org/apache/spark/ml/feature/HivemallLabeledPointSuite.scala +++ /dev/null @@ -1,35 +0,0 @@ -/* - * 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.0/src/test/scala/org/apache/spark/sql/QueryTest.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/QueryTest.scala deleted file mode 100644 index 8b03911ef..000000000 --- a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ /dev/null @@ -1,475 +0,0 @@ -/* - * 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.LogicalRDD -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.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 - } - } - checkJsonFormat(analyzedDS) - 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 - } - } - - checkJsonFormat(analyzedDF) - - 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) - } - - private def checkJsonFormat(ds: Dataset[_]): Unit = { - // Get the analyzed plan and rewrite the PredicateSubqueries in order to make sure that - // RDD and Data resolution does not break. - val logicalPlan = ds.queryExecution.analyzed - - // bypass some cases that we can't handle currently. - logicalPlan.transform { - case _: ObjectConsumer => return - case _: ObjectProducer => return - case _: AppendColumns => return - case _: LogicalRelation => return - case p if p.getClass.getSimpleName == "MetastoreRelation" => return - case _: MemoryPlan => return - }.transformAllExpressions { - case a: ImperativeAggregate => return - case _: TypedAggregateExpression => return - case Literal(_, _: ObjectType) => return - } - - // bypass hive tests before we fix all corner cases in hive module. - if (this.getClass.getName.startsWith("org.apache.spark.sql.hive")) return - - val jsonString = try { - logicalPlan.toJSON - } catch { - case NonFatal(e) => - fail( - s""" - |Failed to parse logical plan to JSON: - |${logicalPlan.treeString} - """.stripMargin, e) - } - - // scala function is not serializable to JSON, use null to replace them so that we can compare - // the plans later. - val normalized1 = logicalPlan.transformAllExpressions { - case udf: ScalaUDF => udf.copy(function = null) - case gen: UserDefinedGenerator => gen.copy(function = null) - } - - // RDDs/data are not serializable to JSON, so we need to collect LogicalPlans that contains - // these non-serializable stuff, and use these original ones to replace the null-placeholders - // in the logical plans parsed from JSON. - val logicalRDDs = new ArrayDeque[LogicalRDD]() - val localRelations = new ArrayDeque[LocalRelation]() - val inMemoryRelations = new ArrayDeque[InMemoryRelation]() - def collectData: (LogicalPlan => Unit) = { - case l: LogicalRDD => - logicalRDDs.offer(l) - case l: LocalRelation => - localRelations.offer(l) - case i: InMemoryRelation => - inMemoryRelations.offer(i) - case p => - p.expressions.foreach { - _.foreach { - case s: SubqueryExpression => - s.query.foreach(collectData) - case _ => - } - } - } - logicalPlan.foreach(collectData) - - - val jsonBackPlan = try { - TreeNode.fromJSON[LogicalPlan](jsonString, spark.sparkContext) - } catch { - case NonFatal(e) => - fail( - s""" - |Failed to rebuild the logical plan from JSON: - |${logicalPlan.treeString} - | - |${logicalPlan.prettyJson} - """.stripMargin, e) - } - - def renormalize: PartialFunction[LogicalPlan, LogicalPlan] = { - case l: LogicalRDD => - val origin = logicalRDDs.pop() - LogicalRDD(l.output, origin.rdd)(spark) - case l: LocalRelation => - val origin = localRelations.pop() - l.copy(data = origin.data) - case l: InMemoryRelation => - val origin = inMemoryRelations.pop() - InMemoryRelation( - l.output, - l.useCompression, - l.batchSize, - l.storageLevel, - origin.child, - l.tableName)( - origin.cachedColumnBuffers, - origin.batchStats) - case p => - p.transformExpressions { - case s: SubqueryExpression => - s.withNewPlan(s.query.transformDown(renormalize)) - } - } - val normalized2 = jsonBackPlan.transformDown(renormalize) - - assert(logicalRDDs.isEmpty) - assert(localRelations.isEmpty) - assert(inMemoryRelations.isEmpty) - - if (normalized1 != normalized2) { - fail( - s""" - |== FAIL: the logical plan parsed from json does not match the original one === - |${sideBySide(logicalPlan.treeString, normalized2.treeString).mkString("\n")} - """.stripMargin) - } - } - - /** - * 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: ${query.queryExecution.analyzed}") - assert(query.queryExecution.optimizedPlan.missingInput.isEmpty, - s"The optimized logical plan has missing inputs: ${query.queryExecution.optimizedPlan}") - assert(query.queryExecution.executedPlan.missingInput.isEmpty, - s"The physical plan has missing inputs: ${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: - |${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.0/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala deleted file mode 100644 index 816576e6a..000000000 --- a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ /dev/null @@ -1,61 +0,0 @@ -/* - * 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.plans.logical.{Filter, LogicalPlan, OneRowRelation} -import org.apache.spark.sql.catalyst.util._ - -/** - * Provides helper methods for comparing plans. - */ -class PlanTest extends SparkFunSuite { - - /** - * 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 a: AttributeReference => - AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(0)) - case a: Alias => - Alias(a.child, a.name)(exprId = ExprId(0)) - } - } - - /** Fails the test if the two plans do not match */ - protected def comparePlans(plan1: LogicalPlan, plan2: LogicalPlan) { - val normalized1 = normalizeExprIds(plan1) - val normalized2 = 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)) - } -} diff --git a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/HiveUdfSuite.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/HiveUdfSuite.scala deleted file mode 100644 index 4a43afcb4..000000000 --- a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/HiveUdfSuite.scala +++ /dev/null @@ -1,160 +0,0 @@ -/* - * 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.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.5.1-incubating-SNAPSHOT") - ) - - // 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.0/src/test/scala/org/apache/spark/sql/hive/HivemallOpsSuite.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/HivemallOpsSuite.scala deleted file mode 100644 index 399a557fb..000000000 --- a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/HivemallOpsSuite.scala +++ /dev/null @@ -1,811 +0,0 @@ -/* - * 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.types._ -import org.apache.spark.test.{TestUtils, VectorQueryTest} -import org.apache.spark.test.TestFPWrapper._ - -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.5.1-incubating-SNAPSHOT")) - } - - test("misc - rowid") { - assert(DummyInputData.select(rowid()).distinct.count == DummyInputData.count) - } - - test("misc - each_top_k") { - import hiveContext.implicits._ - 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") - - // Compute top-1 rows for each group - checkAnswer( - testDf.each_top_k(lit(1), $"key", $"score"), - Row(1, "a", "3", 0.8, Array(2, 5)) :: - Row(1, "b", "4", 0.3, Array(2)) :: - Row(1, "c", "6", 0.3, Array(1, 3)) :: - Nil - ) - - // Compute reverse top-1 rows for each group - checkAnswer( - testDf.each_top_k(lit(-1), $"key", $"score"), - Row(1, "a", "1", 0.5, Array(0, 1, 2)) :: - Row(1, "b", "5", 0.1, Array(3)) :: - Row(1, "c", "6", 0.3, Array(1, 3)) :: - Nil - ) - - // Check if some exceptions thrown in case of some conditions - assert(intercept[AnalysisException] { testDf.each_top_k(lit(0.1), $"key", $"score") } - .getMessage contains "`k` must be integer, however") - assert(intercept[AnalysisException] { testDf.each_top_k(lit(1), $"key", $"data") } - .getMessage contains "must have a comparable type") - } - - test("HIVEMALL-76 top-K funcs must assign the same rank with the rows having the same scores") { - import hiveContext.implicits._ - val testDf = Seq( - ("a", "1", 0.1), - ("b", "5", 0.1), - ("a", "3", 0.1), - ("b", "4", 0.1), - ("a", "2", 0.0) - ).toDF("key", "value", "score") - - // Compute top-1 rows for each group - checkAnswer( - testDf.each_top_k(lit(2), $"key", $"score"), - Row(1, "a", "3", 0.1) :: - Row(1, "a", "1", 0.1) :: - Row(1, "b", "4", 0.1) :: - Row(1, "b", "5", 0.1) :: - 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.0/src/test/scala/org/apache/spark/sql/hive/ModelMixingSuite.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/ModelMixingSuite.scala deleted file mode 100644 index 06a4dc0f3..000000000 --- a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/ModelMixingSuite.scala +++ /dev/null @@ -1,285 +0,0 @@ -/* - * 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.0/src/test/scala/org/apache/spark/sql/hive/XGBoostSuite.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/XGBoostSuite.scala deleted file mode 100644 index 37e0989d1..000000000 --- a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/XGBoostSuite.scala +++ /dev/null @@ -1,143 +0,0 @@ -/* - * 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.types._ -import org.apache.spark.test.VectorQueryTest - -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.startsWith("xgbmodel-")) - } - - 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 => - - // 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 => - - 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 => - - 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.0/src/test/scala/org/apache/spark/sql/hive/benchmark/MiscBenchmark.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/benchmark/MiscBenchmark.scala deleted file mode 100644 index 87782b92d..000000000 --- a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/benchmark/MiscBenchmark.scala +++ /dev/null @@ -1,224 +0,0 @@ -/* - * 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.SparkFunSuite -import org.apache.spark.sql.{Column, DataFrame, Dataset, Row, SparkSession} -import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute -import org.apache.spark.sql.catalyst.encoders.RowEncoder -import org.apache.spark.sql.catalyst.expressions.{EachTopK, Expression, Literal} -import org.apache.spark.sql.catalyst.plans.logical.{Generate, LogicalPlan, Project} -import org.apache.spark.sql.expressions.Window -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.hive.{HiveGenericUDF, HiveGenericUDTF} -import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper -import org.apache.spark.sql.types._ -import org.apache.spark.test.TestUtils -import org.apache.spark.util.Benchmark - -class TestFuncWrapper(df: DataFrame) { - - def each_top_k(k: Column, group: Column, value: Column, args: Column*) - : DataFrame = withTypedPlan { - val clusterDf = df.repartition(group).sortWithinPartitions(group) - Generate(HiveGenericUDTF( - "each_top_k", - new HiveFunctionWrapper("hivemall.tools.EachTopKUDTF"), - (Seq(k, group, value) ++ args).map(_.expr)), - join = false, outer = false, None, - (Seq("rank", "key") ++ args.map(_.named.name)).map(UnresolvedAttribute(_)), - clusterDf.logicalPlan) - } - - def each_top_k_improved(k: Int, group: String, score: String, args: String*) - : DataFrame = withTypedPlan { - val clusterDf = df.repartition(df(group)).sortWithinPartitions(group) - val childrenAttributes = clusterDf.logicalPlan.output - val generator = Generate( - EachTopK( - k, - clusterDf.resolve(group), - clusterDf.resolve(score), - childrenAttributes - ), - join = false, outer = false, None, - (Seq("rank") ++ childrenAttributes.map(_.name)).map(UnresolvedAttribute(_)), - clusterDf.logicalPlan) - val attributes = generator.generatedSet - val projectList = (Seq("rank") ++ args).map(s => attributes.find(_.name == s).get) - Project(projectList, generator) - } - - /** - * 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 { - HiveGenericUDF("sigmoid", - new HiveFunctionWrapper("hivemall.tools.math.SigmoidGenericUDF"), - exprs.map(_.expr)) - } - - /** - * A convenient function to wrap an expression and produce a Column. - */ - @inline private def withExpr(expr: Expression): Column = Column(expr) -} - -class MiscBenchmark extends SparkFunSuite { - - lazy val sparkSession = SparkSession.builder - .master("local[1]") - .appName("microbenchmark") - .config("spark.sql.shuffle.partitions", 1) - .config("spark.sql.codegen.wholeStage", true) - .getOrCreate() - - val numIters = 3 - - private def addBenchmarkCase(name: String, df: DataFrame)(implicit benchmark: Benchmark): Unit = { - benchmark.addCase(name, numIters) { _ => - df.queryExecution.executedPlan(0).execute().foreach(x => Unit) - } - } - - 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 = 100L << 18 - implicit val benchmark = new Benchmark("sigmoid", N) - val schema = StructType( - StructField("value", DoubleType) :: Nil - ) - val testDf = sparkSession.createDataFrame( - sparkSession.range(N).map(_.toDouble).map(Row(_))(RowEncoder(schema)).rdd, - schema - ) - testDf.cache.count // Cached - - 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")) - ) - - addBenchmarkCase( - "closure", - testDf.map { d => - Row(1.0 / (1.0 + Math.exp(-d.getDouble(0)))) - }(RowEncoder(schema)) - ) - - 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") { - /** - * 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 N = 100L << 18 - val topK = 100 - val numGroup = 3 - implicit val benchmark = new Benchmark(s"top-k (k=$topK)", N) - val schema = StructType( - StructField("key", IntegerType) :: - StructField("score", DoubleType) :: - StructField("value", StringType) :: - Nil - ) - val testDf = { - val df = sparkSession.createDataFrame( - sparkSession.sparkContext.range(0, N).map(_.toInt).map { d => - Row(d % numGroup, scala.util.Random.nextDouble(), s"group-${d % numGroup}") - }, - schema - ) - // Test data are clustered by group keys - df.repartition($"key").sortWithinPartitions($"key") - } - testDf.cache.count // Cached - - addBenchmarkCase( - "rank", - testDf.withColumn( - "rank", rank().over(Window.partitionBy($"key").orderBy($"score".desc)) - ).where($"rank" <= topK) - ) - - addBenchmarkCase( - "each_top_k (hive-udf)", - // TODO: If $"value" given, it throws `AnalysisException`. Why? - // testDf.each_top_k(10, $"key", $"score", $"value") - // org.apache.spark.sql.catalyst.analysis.UnresolvedException: Invalid call to name - // on unresolved object, tree: unresolvedalias('value, None) - // at org.apache.spark.sql.catalyst.analysis.UnresolvedAlias.name(unresolved.scala:339) - testDf.each_top_k(lit(topK), $"key", $"score", testDf("value")) - ) - - addBenchmarkCase( - "each_top_k (exprs)", - testDf.each_top_k_improved(topK, "key", "score", "value") - ) - - benchmark.run() - } -} diff --git a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/test/HivemallFeatureQueryTest.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/test/HivemallFeatureQueryTest.scala deleted file mode 100644 index a4733f5d3..000000000 --- a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/test/HivemallFeatureQueryTest.scala +++ /dev/null @@ -1,112 +0,0 @@ -/* - * 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 -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection} -import org.apache.spark.sql.catalyst.expressions.Literal -import org.apache.spark.sql.QueryTest - -/** - * Base class for tests with Hivemall features. - */ -abstract class HivemallFeatureQueryTest extends QueryTest 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.0/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala deleted file mode 100644 index eeb32dced..000000000 --- a/spark/spark-2.0/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.0/src/test/scala/org/apache/spark/streaming/HivemallOpsWithFeatureSuite.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/streaming/HivemallOpsWithFeatureSuite.scala deleted file mode 100644 index b15c77c4c..000000000 --- a/spark/spark-2.0/src/test/scala/org/apache/spark/streaming/HivemallOpsWithFeatureSuite.scala +++ /dev/null @@ -1,154 +0,0 @@ -/* - * 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.0/src/test/scala/org/apache/spark/test/TestUtils.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/test/TestUtils.scala deleted file mode 100644 index 8a2a385c0..000000000 --- a/spark/spark-2.0/src/test/scala/org/apache/spark/test/TestUtils.scala +++ /dev/null @@ -1,64 +0,0 @@ -/* - * 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) - } -} diff --git a/spark/spark-2.0/src/test/scala/org/apache/spark/test/VectorQueryTest.scala b/spark/spark-2.0/src/test/scala/org/apache/spark/test/VectorQueryTest.scala deleted file mode 100644 index e1384d178..000000000 --- a/spark/spark-2.0/src/test/scala/org/apache/spark/test/VectorQueryTest.scala +++ /dev/null @@ -1,89 +0,0 @@ -/* - * 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 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 TestHiveSingleton { - import hiveContext.implicits._ - - 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) - } - } -}