diff --git a/DEVEL.md b/DEVEL.md index 0a530b937..eb03912ae 100644 --- a/DEVEL.md +++ b/DEVEL.md @@ -193,12 +193,6 @@ val sc:SparkContext = ... val hc = H2OContext.getOrCreate(sc) ``` -or: -```scala -val sc:SparkContext = ... -val hc = new H2OContext(sc).start() -``` - The call will: 1. Collect the number and host names of the executors (worker nodes) in the Spark cluster 2. Launch H2O services on each detected executor @@ -226,39 +220,58 @@ The environment must contain the property `SPARK_HOME` that points to the Spark ### Sparkling Water Configuration Properties -The following configuration properties can be passed to Spark to configure Sparking Water: +The following configuration properties can be passed to Spark to configure Sparking Water +####Configuration properties independent on selected backend | Property name | Default value | Description | |---------------|---------------|-------------| | **Generic parameters** ||| -|`spark.ext.h2o.flatfile` | `true`| Use flatfile (instead of multicast) approach for creating H2O cloud | -|`spark.ext.h2o.cluster.size` | `-1` |Expected number of workers of H2O cloud. Use -1 to automatically detect the cluster size. This number must be equal to number of Spark workers.| -|`spark.ext.h2o.port.base`| `54321`| Base port used for individual H2O node configuration.| -|`spark.ext.h2o.port.incr`| `2` | Increment added to base port to find the next available port.| -|`spark.ext.h2o.cloud.timeout`| `60*1000` | Timeout (in msec) for cloud | -|`spark.ext.h2o.spreadrdd.retries` | `10` | Number of retries for creation of an RDD covering all existing Spark executors. | -|`spark.ext.h2o.cloud.name`| `sparkling-water-` | Name of H2O cloud. | +|`spark.ext.h2o.cloud.name`| `sparkling-water-` | Name of H2O cloud.| |`spark.ext.h2o.nthreads`|`-1`|Limit for number of threads used by H2O, default `-1` means unlimited.| |`spark.ext.h2o.disable.ga`|`false`|Disable Google Analytics tracking for embedded H2O.| -|`spark.ext.h2o.subseq.tries`|`5`|Subsequent successful tries to figure out size of Spark cluster which are producing the same number of nodes.| -|`spark.ext.h2o.exit.on.unsupported.spark.param`|`true`|If unsupported Spark parameters is detected, then application is forced to shutdown.| -|`spark.ext.h2o.topology.change.listener.enabled`|`true`|Decides whether listener which kills h2o cloud on the change of underlying cluster's topology is enabled or not.| -|`spark.ext.h2o.spark.version.check.enabled`|`true`|Enables check if runtime Spark version matches build time Spark version.| |`spark.ext.h2o.repl.enabled`|`true`|Decides whether H2O repl is initialized or not. The repl is initialized by default.| |`spark.ext.scala.int.default.num`|`1`|Number of executors started at the start of h2o services.| -| **H2O server node parameters** ||| -|`spark.ext.h2o.node.log.level`| `INFO`| H2O internal log level used for launched H2O nodes. | -|`spark.ext.h2o.node.log.dir`| ` System.getProperty("user.dir") + File.separator + "h2ologs"` or YARN container dir| Location of h2o logs on executor machine. | -|`spark.ext.h2o.node.network.mask`|--|Subnet selector for H2O running inside Spark executors, this disables using IP reported by Spark but tries to find IP based on the specified mask.| -| **H2O client parameters** ||| -|`spark.ext.h2o.client.log.level`| `INFO`| H2O internal log level used for H2O client running inside Spark driver. | -|`spark.ext.h2o.client.log.dir`| ` System.getProperty("user.dir") + File.separator + "h2ologs"`| Location of h2o logs on driver machine. | +|`spark.ext.h2o.topology.change.listener.enabled`|`true`|Decides whether listener which kills h2o cloud on the change of underlying cluster's topology is enabled or not.| +|`spark.ext.h2o.spark.version.check.enabled`|`true`|Enables check if runtime Spark version matches build time Spark version.| +|`spark.ext.h2o.exit.on.unsupported.spark.param`|`true`|If unsupported Spark parameters is detected, then application is forced to shutdown.| +|`spark.ext.h2o.jks`|`null`|Path to Java KeyStore file.| +|`spark.ext.h2o.jks.pass`|`null`|Password for Java KeyStore file.| +|`spark.ext.h2o.hash.login`|`false`|Enable hash login.| +|`spark.ext.h2o.ldap.login`|`false`|Enable LDAP login.| +|`spark.ext.h2o.kerberos.login`|`false`|Enable Kerberos login.| +|`spark.ext.h2o.login.conf`|`null`|Login configuration file.| +|`spark.ext.h2o.user.name`|`null`|Override user name for cluster.| +| **H2O client parameters** ||| +|`spark.ext.h2o.client.ip`|`null`|IP of H2O client node | +|`spark.ext.h2o.client.iced.dir`|`null`|Location of iced directory for the driver instance.| +|`spark.ext.h2o.client.log.level`| `INFO`| H2O internal log level used for H2O client running inside Spark driver.| +|`spark.ext.h2o.client.log.dir`| ` System.getProperty("user.dir") + File.separator + "h2ologs"`| Location of h2o logs on driver machine.| +|`spark.ext.h2o.client.port.base`|`54321`| Port on which H2O client publishes its API. If already occupied, the next odd port is tried and so on.| |`spark.ext.h2o.client.web.port`|`-1`|Exact client port to access web UI. The value `-1` means automatic search for free port starting at `spark.ext.h2o.port.base`.| |`spark.ext.h2o.client.verbose`|`false`|The client outputs verbosed log output directly into console. Enabling the flag increases the client log level to INFO.| |`spark.ext.h2o.client.network.mask`|--|Subnet selector for H2O client, this disables using IP reported by Spark but tries to find IP based on the specifed mask.| +--- +####Internal backend configuration properties +| Property name | Default value | Description | +|---------------|---------------|-------------| +| **Generic parameters** ||| +|`spark.ext.h2o.flatfile` | `true`|Use flatfile (instead of multicast) approach for creating H2O cloud.| +|`spark.ext.h2o.cluster.size`| `-1` |Expected number of workers of H2O cloud. Use -1 to automatically detect the cluster size. This number must be equal to number of Spark workers.| +|`spark.ext.h2o.port.base`| `54321`| Base port used for individual H2O node configuration.| +|`spark.ext.h2o.cloud.timeout`| `60*1000` |Timeout (in msec) for cloud.| +|`spark.ext.h2o.dummy.rdd.mul.factor`| `10`|Multiplication factor for dummy RDD generation.
Size of dummy RDD is spark.ext.h2o.cluster.size*spark.ext.h2o.dummy.rdd.mul.factor.| +|`spark.ext.h2o.spreadrdd.retries`| `10` |Number of retries for creation of an RDD covering all existing Spark executors.| +|`spark.ext.h2o.default.cluster.size`| `20`|Starting size of cluster in case that size is not explicitly passed.| +|`spark.ext.h2o.node.iced.dir`| `null`|Location of iced directory for Spark nodes.| +|`spark.ext.h2o.subseq.tries`|`5`|Subsequent successful tries to figure out size of Spark cluster which are producing the same number of nodes.| +| **H2O server node parameters** ||| +|`spark.ext.h2o.node.network.mask`|--|Subnet selector for H2O running inside Spark executors, this disables using IP reported by Spark but tries to find IP based on the specified mask.| +|`spark.ext.h2o.node.log.level`| `INFO`| H2O internal log level used for launched H2O nodes.| +|`spark.ext.h2o.node.log.dir`| ` System.getProperty("user.dir") + File.separator + "h2ologs"` or YARN container dir| Location of h2o logs on executor machine.| --- + # Running Sparkling Water @@ -271,24 +284,6 @@ val sc:SparkContext = ... val hc = H2OContext.getOrCreate(sc) ``` -or: -```scala -val sc:SparkContext = ... -val hc = new H2OContext(sc).start() -``` - -When the number of Spark nodes is known, it can be specified in `getOrCreate` call: -```scala -val hc = H2OContext.getOrCreate(sc, numOfSparkNodes) -``` - -or in `start` method of H2O Context: -```scala -val hc = new H2OContext(sc).start(numOfSparkNodes) -``` - -The former variant is preferred, because it initiates and starts H2O Context in one call and also can be used to obtain already existing H2OContext, but it does semantically the same as the latter variant. - --- ### Memory Allocation diff --git a/README.md b/README.md index 3a01c7e80..f69b320ee 100644 --- a/README.md +++ b/README.md @@ -98,6 +98,37 @@ There are several ways of using Sparkling Water: - pySpark with pySparkling --- + +H2O cloud is created automatically during the call of `H2OContext.getOrCreate`. Since it's not +technically possible to get number of executors in Spark, we try to discover all executors at the initiation of `H2OContext` +and we start H2O instance inside of each discovered executor. This solution is easiest to deploy; however when Spark +or YARN kills the executor - which is not an unusual case - the whole H2O cluster goes down since h2o doesn't support high +availability. + + +Here we show a few examples how H2OContext can be started. + +Explicitly specify internal backend on `H2OConf` +``` +val conf = new H2OConf(sc).setInternalClusterMode() +val h2oContext = H2OContext.getOrCreate(sc, conf) +``` + +If `spark.ext.h2o.backend.cluster.mode` property was set to `internal` either on command line or on the `SparkConf` class + we can call: +``` +val h2oContext = H2OContext.getOrCreate(sc) +``` + +or + +``` +val conf = new H2OConf(sc) +val h2oContext = H2OContext.getOrCreate(sc, conf) +``` + + + ### Run Sparkling shell diff --git a/core/build.gradle b/core/build.gradle index c13b8e1d3..c8678e654 100644 --- a/core/build.gradle +++ b/core/build.gradle @@ -25,7 +25,8 @@ dependencies { exclude(group: "com.fasterxml.jackson.core") } - // Spark 1.2.0 release + // Spark components + // - core compile "org.apache.spark:spark-core_${scalaBaseVersion}:${sparkVersion}" // - SQL component @@ -57,8 +58,7 @@ dependencies { // Setup test environment for Spark test { // Test environment - systemProperty "spark.testing", "true" - + systemProperty "spark.testing", "true" systemProperty "spark.ext.h2o.node.log.dir", new File(project.getBuildDir(), "h2ologs-test/nodes") systemProperty "spark.ext.h2o.client.log.dir", new File(project.getBuildDir(), "h2ologs-test/client") diff --git a/core/src/integTest/scala/water/sparkling/itest/local/H2OContextLocalClusterSuite.scala b/core/src/integTest/scala/water/sparkling/itest/local/H2OContextLocalClusterSuite.scala index 4dac02cea..de011b124 100644 --- a/core/src/integTest/scala/water/sparkling/itest/local/H2OContextLocalClusterSuite.scala +++ b/core/src/integTest/scala/water/sparkling/itest/local/H2OContextLocalClusterSuite.scala @@ -17,8 +17,8 @@ package water.sparkling.itest.local import org.apache.spark.SparkContext -import org.apache.spark.h2o.H2OContext -import org.apache.spark.h2o.util.SparkTestContext +import org.apache.spark.h2o.{H2OConf, H2OContext} +import org.apache.spark.h2o.utils.SparkTestContext import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} @@ -37,7 +37,7 @@ class H2OContextLocalClusterSuite extends FunSuite // For distributed testing we need to pass around jar containing all implementation classes plus test classes val conf = defaultSparkConf.setJars(swassembly :: Nil) sc = new SparkContext("local-cluster[3,2,2048]", "test-local-cluster", conf) - hc = H2OContext.getOrCreate(sc) + hc = H2OContext.getOrCreate(sc, new H2OConf(sc)) assert(water.H2O.CLOUD.members().length == 3, "H2O cloud should have 3 members") // Does not reset diff --git a/core/src/integTest/scala/water/sparkling/itest/local/H2OContextLocalSuite.scala b/core/src/integTest/scala/water/sparkling/itest/local/H2OContextLocalSuite.scala index 52102cff0..7f3a6ba50 100644 --- a/core/src/integTest/scala/water/sparkling/itest/local/H2OContextLocalSuite.scala +++ b/core/src/integTest/scala/water/sparkling/itest/local/H2OContextLocalSuite.scala @@ -17,8 +17,8 @@ package water.sparkling.itest.local import org.apache.spark.SparkContext -import org.apache.spark.h2o.H2OContext -import org.apache.spark.h2o.util.SparkTestContext +import org.apache.spark.h2o.{H2OConf, H2OContext} +import org.apache.spark.h2o.utils.SparkTestContext import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} @@ -34,7 +34,7 @@ class H2OContextLocalSuite extends FunSuite test("verify H2O cloud building on local JVM") { sc = new SparkContext("local[*]", "test-local", defaultSparkConf) - hc = H2OContext.getOrCreate(sc) + hc = H2OContext.getOrCreate(sc, new H2OConf(sc)) // Number of nodes should be on assert(water.H2O.CLOUD.members().length == 1, "H2O cloud should have 1 members") // Make sure that H2O is running diff --git a/core/src/main/scala/org/apache/spark/h2o/DefaultSource.scala b/core/src/main/scala/org/apache/spark/h2o/DefaultSource.scala index d10a9724f..22d91f32a 100644 --- a/core/src/main/scala/org/apache/spark/h2o/DefaultSource.scala +++ b/core/src/main/scala/org/apache/spark/h2o/DefaultSource.scala @@ -17,20 +17,17 @@ package org.apache.spark.h2o -import org.apache.spark.sql.{DataFrame, H2OFrameRelation, SQLContext, SaveMode} import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{DataFrame, H2OFrameRelation, SQLContext, SaveMode} import water.{DKV, Key} /** * Provides access to H2OFrame from pure SQL statements (i.e. for users of the * JDBC server). */ -class DefaultSource - extends RelationProvider - with SchemaRelationProvider - with CreatableRelationProvider - with DataSourceRegister{ +class DefaultSource extends RelationProvider + with SchemaRelationProvider with CreatableRelationProvider with DataSourceRegister{ /** * Short alias for spark-csv data source. @@ -50,9 +47,8 @@ class DefaultSource * Creates a new relation for data store in H2OFrame given parameters. * Parameters have to include 'key' */ - override def createRelation( - sqlContext: SQLContext, - parameters: Map[String, String]): BaseRelation = { + override def createRelation(sqlContext: SQLContext, + parameters: Map[String, String]): BaseRelation = { createRelation(sqlContext, parameters, null) } @@ -60,9 +56,9 @@ class DefaultSource * Creates a new relation for data store in H2OFrame given parameters. * Parameters have to include 'key' */ - override def createRelation( sqlContext: SQLContext, - parameters: Map[String, String], - schema: StructType): H2OFrameRelation[_] = { + override def createRelation(sqlContext: SQLContext, + parameters: Map[String, String], + schema: StructType): H2OFrameRelation[_] = { val key = checkKey(parameters) H2OFrameRelation(getFrame(key), true)(sqlContext) @@ -73,11 +69,14 @@ class DefaultSource parameters: Map[String, String], data: DataFrame): BaseRelation = { val key = checkKey(parameters) - val originalFrame = DKV.getGet[H2OFrame](key) - - implicit val h2oContext = H2OContext.get().getOrElse(throw new RuntimeException("H2OContext has to be started in" + - " oder to save/load frames using H2O Data source")) - + val originalFrame = DKV.getGet[Frame](key) + implicit val h2oContext = { + if(H2OContext.get().isEmpty){ + throw new RuntimeException("H2OContext has to be started in order to save/load data using H2O Data source.") + }else{ + H2OContext.get().get + } + } if(originalFrame!=null){ mode match { case SaveMode.Append => @@ -90,7 +89,7 @@ class DefaultSource } } else { // save as H2O Frame - h2oContext.asH2OFrame(data,key) + h2oContext.asH2OFrame(data, key) } createRelation(sqlContext, parameters, data.schema) diff --git a/core/src/main/scala/org/apache/spark/h2o/H2OConf.scala b/core/src/main/scala/org/apache/spark/h2o/H2OConf.scala index a3f595505..5e3f44a16 100644 --- a/core/src/main/scala/org/apache/spark/h2o/H2OConf.scala +++ b/core/src/main/scala/org/apache/spark/h2o/H2OConf.scala @@ -17,237 +17,76 @@ package org.apache.spark.h2o -import org.apache.spark.SparkConf -import org.apache.spark.h2o.H2OConf._ +import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.h2o.backends.internal.InternalBackendConf +import org.apache.spark.{Logging, SparkContext} /** - * Just simple configuration holder which is representing - * properties passed from user to H2O App. - */ -trait H2OConf { - - /* Require Spar config */ - def sparkConf: SparkConf + * Configuration holder which is representing + * properties passed from user to Sparkling Water. + */ +class H2OConf(@transient val sc: SparkContext) extends Logging with InternalBackendConf { + + /** Support for creating H2OConf in Java environments */ + def this(jsc: JavaSparkContext) = this(jsc.sc) + val sparkConf = sc.getConf // Precondition - require(sparkConf != null, "sparkConf was null") - - /* Initialize configuration */ - // Collect configuration properties - import H2OConf._ - - def numH2OWorkers = sparkConf.getOption(PROP_CLUSTER_SIZE._1).map(_.toInt) - def useFlatFile = sparkConf.getBoolean(PROP_USE_FLATFILE._1, PROP_USE_FLATFILE._2) - def clientIp = sparkConf.getOption(PROP_CLIENT_IP._1) - def clientBasePort = sparkConf.getInt(PROP_CLIENT_PORT_BASE._1, PROP_CLIENT_PORT_BASE._2) - def nodeBasePort = sparkConf.getInt(PROP_NODE_PORT_BASE._1, PROP_NODE_PORT_BASE._2) - def cloudTimeout = sparkConf.getInt(PROP_CLOUD_TIMEOUT._1, PROP_CLOUD_TIMEOUT._2) - def drddMulFactor = sparkConf.getInt(PROP_DUMMY_RDD_MUL_FACTOR._1, PROP_DUMMY_RDD_MUL_FACTOR._2) - def numRddRetries = sparkConf.getInt(PROP_SPREADRDD_RETRIES._1, PROP_SPREADRDD_RETRIES._2) - def cloudName = sparkConf.get(PROP_CLOUD_NAME._1, PROP_CLOUD_NAME._2) - def defaultCloudSize = sparkConf.getInt(PROP_DEFAULT_CLUSTER_SIZE._1, PROP_DEFAULT_CLUSTER_SIZE._2) - def h2oNodeLogLevel = sparkConf.get(PROP_NODE_LOG_LEVEL._1, PROP_NODE_LOG_LEVEL._2) - def h2oClientLogLevel = sparkConf.get(PROP_CLIENT_LOG_LEVEL._1, PROP_CLIENT_LOG_LEVEL._2) - def h2oNodeLogDir = sparkConf.get(PROP_NODE_LOG_DIR._1, PROP_NODE_LOG_DIR._2) - def h2oClientLogDir = sparkConf.get(PROP_CLIENT_LOG_DIR._1, PROP_CLIENT_LOG_DIR._2) - def clientNetworkMask = sparkConf.getOption(PROP_CLIENT_NETWORK_MASK._1) - def nodeNetworkMask = sparkConf.getOption(PROP_NODE_NETWORK_MASK._1) - def nthreads = sparkConf.getInt(PROP_NTHREADS._1, PROP_NTHREADS._2) - def disableGA = sparkConf.getBoolean(PROP_DISABLE_GA._1, PROP_DISABLE_GA._2) - def clientWebPort = sparkConf.getInt(PROP_CLIENT_WEB_PORT._1, PROP_CLIENT_WEB_PORT._2) - def clientIcedDir = sparkConf.getOption(PROP_CLIENT_ICED_DIR._1) - def clientVerboseOutput = sparkConf.getBoolean(PROP_CLIENT_VERBOSE._1, PROP_CLIENT_VERBOSE._2) - def nodeIcedDir = sparkConf.getOption(PROP_NODE_ICED_DIR._1) - - def jks = sparkConf.getOption(PROP_JKS._1) - def jksPass = sparkConf.getOption(PROP_JKS_PASS._1) - def hashLogin = sparkConf.getBoolean(PROP_HASH_LOGIN._1, PROP_HASH_LOGIN._2) - def ldapLogin = sparkConf.getBoolean(PROP_LDAP_LOGIN._1, PROP_LDAP_LOGIN._2) - def kerberosLogin = sparkConf.getBoolean(PROP_KERBEROS_LOGIN._1, PROP_KERBEROS_LOGIN._2) - def loginConf = sparkConf.getOption(PROP_LOGIN_CONF._1) - def userName = sparkConf.getOption(PROP_USER_NAME._1) - - def subseqTries = sparkConf.getInt(PROP_SUBSEQ_TRIES._1, PROP_SUBSEQ_TRIES._2) - def scalaIntDefaultNum = sparkConf.getInt(PROP_SCALA_INT_DEFAULT_NUM._1, PROP_SCALA_INT_DEFAULT_NUM._2) - def isH2OReplEnabled = sparkConf.getBoolean(PROP_REPL_ENABLED._1,PROP_REPL_ENABLED._2) - def isClusterTopologyListenerEnabled = sparkConf.getBoolean(PROP_CLUSTER_TOPOLOGY_LISTENER_ENABLED._1,PROP_CLUSTER_TOPOLOGY_LISTENER_ENABLED._2) - def isSparkVersionCheckEnabled = sparkConf.getBoolean(PROP_SPARK_VERSION_CHECK_ENABLED._1,PROP_SPARK_VERSION_CHECK_ENABLED._2) - def isFailOnUnsupportedSparkParamEnabled = sparkConf.getBoolean(PROP_FAIL_ON_UNSUPPORTED_SPARK_PARAM._1, PROP_FAIL_ON_UNSUPPORTED_SPARK_PARAM._2) - - /** - * Produce arguments for H2O node based on this config. - * - * @return array of H2O launcher command line arguments - */ - def getH2ONodeArgs: Array[String] = (getH2OCommonOptions ++ - Seq("-log_level", h2oNodeLogLevel, - "-baseport", nodeBasePort.toString)).toArray - - /** - * Get arguments for H2O client. - * - * @return array of H2O client arguments. - */ - - def getH2OClientArgs(ip: String): Array[String] = ( - getH2OCommonOptions - ++ Seq("-client") - ++ (if (!clientVerboseOutput) Seq("-quiet") else Nil) - ++ (if (hashLogin) Seq("-hash_login") else Nil) - ++ (if (ldapLogin) Seq("-ldap_login") else Nil) - ++ (if (kerberosLogin) Seq("-kerberos_login") else Nil) - ++ Seq("-log_level", if (clientVerboseOutput) incLogLevel(h2oClientLogLevel, "INFO") else h2oClientLogLevel) - ++ Seq("-log_dir", h2oClientLogDir) - ++ Seq("-baseport", clientBasePort.toString) - ++ Seq( - ("-ice_root", clientIcedDir.orNull), - ("-port", if (clientWebPort > 0) clientWebPort else null), - ("-jks", jks.orNull), - ("-jks_pass", jksPass.orNull), - ("-login_conf", loginConf.orNull), - ("-user_name", userName.orNull), - clientNetworkMask.map(mask => ("-network", mask)).getOrElse(("-ip", ip)) - ).filter(_._2 != null).flatMap(x => Seq(x._1, x._2.toString)) - ).toArray - - private def getH2OCommonOptions: Seq[String] = - // Option in form key=value - Seq( - ("-name", cloudName), - ("-nthreads", if (nthreads > 0) nthreads else null)) - .filter(x => x._2 != null) - .flatMap(x => Seq(x._1, x._2.toString)) ++ // Append single boolean options - Seq(("-ga_opt_out", disableGA)) - .filter(_._2).map(x => x._1) - - /** - * Returns Major Spark version for which is this version of Sparkling Water designated. - * - * For example, for 1.6.1 returns 1.6 - */ - def buildSparkMajorVersion = { - val stream = getClass.getResourceAsStream("/spark.version") - val version = scala.io.Source.fromInputStream(stream).mkString - if (version.count(_ == '.') == 1) { // e.g., 1.6 - version - } else { // 1.4 - version.substring(0, version.lastIndexOf('.')) - } + require(sparkConf != null, "Spark conf was null") + + /** Copy this object */ + override def clone: H2OConf = { + new H2OConf(sc).setAll(getAll) + } + + /** Set a configuration variable. */ + def set(key: String, value: String): H2OConf = { + sparkConf.set(key, value) + this + } + + /** Remove a parameter from the configuration */ + def remove(key: String): H2OConf = { + sparkConf.remove(key) + this + } + + def contains(key: String): Boolean = sparkConf.contains(key) + + /** Get a parameter; throws a NoSuchElementException if it's not set */ + def get(key: String): String = sparkConf.get(key) + + /** Get a parameter, falling back to a default if not set */ + def get(key: String, defaultValue: String): String = sparkConf.get(key, defaultValue) + + /** Get a parameter as an Option */ + def getOption(key: String): Option[String] = sparkConf.getOption(key) + + /** Get all parameters as a list of pairs */ + def getAll: Array[(String, String)] = { + sparkConf.getAll } - override def toString: String = - s"""Sparkling Water configuration: - | workers : $numH2OWorkers - | cloudName : $cloudName - | flatfile : $useFlatFile - | clientBasePort : $clientBasePort - | nodeBasePort : $nodeBasePort - | cloudTimeout : $cloudTimeout - | h2oNodeLog : $h2oNodeLogLevel - | h2oClientLog : $h2oClientLogLevel - | nthreads : $nthreads - | drddMulFactor : $drddMulFactor""".stripMargin - - /** - * Increment log level to at least desired minimal log level. - * @param logLevel actual log level - * @param minLogLevel desired minimal log level - * @return if logLevel is less verbose than minLogLeve then minLogLevel, else logLevel - */ - private def incLogLevel(logLevel: String, minLogLevel: String): String = { - val logLevels = Seq( ("OFF", 0), ("FATAL", 1), ("ERROR", 2), - ("WARN", 3), ("INFO", 4), ("DEBUG", 5), ("TRACE", 6), ("ALL", 7)) - val ll = logLevels.find(t => t._1 == logLevel) - val mll = logLevels.find(t => t._1 == minLogLevel) - if (mll.isEmpty) { - logLevel - } else { - ll.map(v => if (v._2 < mll.get._2) minLogLevel else logLevel).getOrElse(minLogLevel) - } + /** Set multiple parameters together */ + def setAll(settings: Traversable[(String, String)]): H2OConf = { + sparkConf.setAll(settings) + this } + + /** Get a parameter as an integer, falling back to a default if not set */ + def getInt(key: String, defaultValue: Int): Int = sparkConf.getInt(key, defaultValue) + + /** Get a parameter as a long, falling back to a default if not set */ + def getLong(key: String, defaultValue: Long): Long = sparkConf.getLong(key, defaultValue) + + /** Get a parameter as a double, falling back to a default if not set */ + def getDouble(key: String, defaultValue: Double): Double = sparkConf.getDouble(key, defaultValue) + + /** Get a parameter as a boolean, falling back to a default if not set */ + def getBoolean(key: String, defaultValue: Boolean): Boolean = sparkConf.getBoolean(key, defaultValue) + + override def toString: String = internalConfString } object H2OConf { - /* Configuration properties */ - - /** Configuration property - use flatfile for H2O cloud formation. */ - val PROP_USE_FLATFILE = ("spark.ext.h2o.flatfile", true) - /** Configuration property - expected number of workers of H2O cloud. - * Value None means automatic detection of cluster size. - */ - val PROP_CLUSTER_SIZE = ("spark.ext.h2o.cluster.size", None) - /** Configuration property - timeout for cloud up. */ - val PROP_CLOUD_TIMEOUT = ("spark.ext.h2o.cloud.timeout", 60*1000) - /** Configuration property - number of retries to create an RDD spreat over all executors */ - val PROP_SPREADRDD_RETRIES = ("spark.ext.h2o.spreadrdd.retries", 10) - /** Configuration property - name of H2O cloud */ - val PROP_CLOUD_NAME = ("spark.ext.h2o.cloud.name", "sparkling-water-") - /** Starting size of cluster in case that size is not explicitelly passed */ - val PROP_DEFAULT_CLUSTER_SIZE = ("spark.ext.h2o.default.cluster.size", 20) - /* H2O internal log level for launched remote nodes. */ - val PROP_NODE_LOG_LEVEL = ("spark.ext.h2o.node.log.level", "INFO") - /** H2O log leve for client running in Spark driver */ - val PROP_CLIENT_LOG_LEVEL = ("spark.ext.h2o.client.log.level", "WARN") - /** Location of log directory for remote nodes. */ - val PROP_NODE_LOG_DIR = ("spark.ext.h2o.node.log.dir", null.asInstanceOf[String]) - /** Location of log directory for the driver instance. */ - val PROP_CLIENT_LOG_DIR = ("spark.ext.h2o.client.log.dir", defaultLogDir) - /** Subnet selector for H2O client - if the mask is specified then Spark network setup is not discussed. */ - val PROP_CLIENT_NETWORK_MASK = ("spark.ext.h2o.client.network.mask", null.asInstanceOf[String]) - /** Subnet selector for H2O nodes running inside executors - if the mask is specified then Spark network setup is not discussed. */ - val PROP_NODE_NETWORK_MASK = ("spark.ext.h2o.node.network.mask", null.asInstanceOf[String]) - /** Limit for number of threads used by H2O, default -1 means unlimited */ - val PROP_NTHREADS = ("spark.ext.h2o.nthreads", -1) - /** Disable GA tracking */ - val PROP_DISABLE_GA = ("spark.ext.h2o.disable.ga", true) - /** Exact client port to access web UI. - * The value `-1` means automatic search for free port starting at `spark.ext.h2o.port.base`. */ - val PROP_CLIENT_WEB_PORT = ("spark.ext.h2o.client.web.port", -1) - /** Location of iced directory for the driver instance. */ - val PROP_CLIENT_ICED_DIR = ("spark.ext.h2o.client.iced.dir", null.asInstanceOf[String]) - /** Configuration property - base port used for individual H2O nodes configuration. */ - val PROP_NODE_PORT_BASE = ( "spark.ext.h2o.node.port.base", 54321 ) - /** Base port for finding non-allocated port for H2O client. */ - val PROP_CLIENT_PORT_BASE = ( "spark.ext.h2o.client.port.base", 54321 ) - /** Explicit client IP */ - val PROP_CLIENT_IP = ("spark.ext.h2o.client.ip", null.asInstanceOf[String]) - /** Print detailed messages to client stdout */ - val PROP_CLIENT_VERBOSE = ("spark.ext.h2o.client.verbose", false) - - /** Location of iced directory for Spark nodes */ - val PROP_NODE_ICED_DIR = ("spark.ext.h2o.node.iced.dir", null.asInstanceOf[String]) - - /** Configuration property - multiplication factor for dummy RDD generation. - * Size of dummy RDD is PROP_CLUSTER_SIZE*PROP_DUMMY_RDD_MUL_FACTOR */ - val PROP_DUMMY_RDD_MUL_FACTOR = ("spark.ext.h2o.dummy.rdd.mul.factor", 10) - - /** Path to Java KeyStore file. */ - val PROP_JKS = ("spark.ext.h2o.jks", null.asInstanceOf[String]) - /** Password for Java KeyStore file. */ - val PROP_JKS_PASS = ("spark.ext.h2o.jks.pass", null.asInstanceOf[String]) - /** Enable hash login. */ - val PROP_HASH_LOGIN = ("spark.ext.h2o.hash.login", false) - /** Enable LDAP login. */ - val PROP_LDAP_LOGIN = ("spark.ext.h2o.ldap.login", false) - /** Enable Kerberos login. */ - val PROP_KERBEROS_LOGIN = ("spark.ext.h2o.kerberos.login", false) - /** Login configuration file. */ - val PROP_LOGIN_CONF = ("spark.ext.h2o.login.conf", null.asInstanceOf[String]) - /** Override user name for cluster. */ - val PROP_USER_NAME = ("spark.ext.h2o.user.name", null.asInstanceOf[String]) - /** Subsequent successful tries to figure out size of Spark cluster which are producing same number of nodes. */ - val PROP_SUBSEQ_TRIES = ("spark.ext.h2o.subseq.tries", 5) - /** Number of executors started at the start of h2o services, by default 1 */ - val PROP_SCALA_INT_DEFAULT_NUM = ("spark.ext.scala.int.default.num",1) - /** Enable/Disable Sparkling-Water REPL **/ - val PROP_REPL_ENABLED = ("spark.ext.h2o.repl.enabled",true) - /** Enable/Disable listener which kills H2O when there is a change in underlying cluster's topology**/ - val PROP_CLUSTER_TOPOLOGY_LISTENER_ENABLED = ("spark.ext.h2o.topology.change.listener.enabled",true) - /** Enable/Disable check for Spark version. */ - val PROP_SPARK_VERSION_CHECK_ENABLED = ("spark.ext.h2o.spark.version.check.enabled",true) - /** Enable/Disable exit on unsupported Spark parameters. */ - val PROP_FAIL_ON_UNSUPPORTED_SPARK_PARAM = ("spark.ext.h2o.fail.on.unsupported.spark.param",true) - - private[spark] def defaultLogDir: String = { - System.getProperty("user.dir") + java.io.File.separator + "h2ologs" - } + def apply(sc: SparkContext) = new H2OConf(sc) } diff --git a/core/src/main/scala/org/apache/spark/h2o/H2OContext.scala b/core/src/main/scala/org/apache/spark/h2o/H2OContext.scala index fdc6ab3a8..ded46a59e 100644 --- a/core/src/main/scala/org/apache/spark/h2o/H2OContext.scala +++ b/core/src/main/scala/org/apache/spark/h2o/H2OContext.scala @@ -20,59 +20,104 @@ package org.apache.spark.h2o import java.util.concurrent.atomic.AtomicReference import org.apache.spark._ -import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} -import org.apache.spark.h2o.H2OContextUtils._ -import org.apache.spark.h2o.H2OTypeUtils._ -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.rdd.H2ORDD -import org.apache.spark.repl.SparkIMain -import org.apache.spark.scheduler.{SparkListener, SparkListenerExecutorAdded} -import org.apache.spark.sql.types._ -import org.apache.spark.sql.{DataFrame, H2OFrameRelation, Row, SQLContext} +import org.apache.spark.h2o.backends.SparklingBackend +import org.apache.spark.h2o.backends.internal.InternalH2OBackend +import org.apache.spark.h2o.converters._ +import org.apache.spark.h2o.utils.{H2OContextUtils, NodeDesc} +import org.apache.spark.sql.{DataFrame, SQLContext} import water._ -import water.api.DataFrames.DataFramesHandler -import water.api.H2OFrames.H2OFramesHandler -import water.api.RDDs.RDDsHandler -import water.api._ -import water.api.scalaInt.ScalaCodeHandler -import water.parser.BufferedString import scala.collection.mutable -import scala.collection.mutable.ListBuffer import scala.language.implicitConversions import scala.reflect.ClassTag import scala.reflect.runtime.universe._ -import scala.util.Random import scala.util.control.NoStackTrace /** - * Simple H2O context motivated by SQLContext. - * - * It provides implicit conversion from RDD -> H2OLikeRDD and back. + * Main entry point for Sparkling Water functionality. H2O Context represents connection to H2O cluster and allows as + * to operate with it. + * + * H2O Context provides conversion methods from RDD/DataFrame to H2OFrame and back and also provides implicits + * conversions for desired transformations + * + * Sparkling Water can run in two modes. External cluster mode and internal cluster mode. When using external cluster + * mode, it tries to connect to existing H2O cluster using the provided spark + * configuration properties. In the case of internal cluster mode,it creates H2O cluster living in Spark - that means + * that each Spark executor will have one h2o instance running in it. This mode is not + * recommended for big clusters and clusters where Spark executors are not stable. + * + * Cluster mode can be set using the spark configuration + * property spark.ext.h2o.mode which can be set in script starting sparkling-water or + * can be set in H2O configuration class H2OConf */ -class H2OContext (@transient val sparkContext: SparkContext) extends { - val sparkConf = sparkContext.getConf - } with org.apache.spark.Logging - with H2OConf - with Serializable { - self => - /** Supports call from java environments. */ - def this(sparkContext: JavaSparkContext) = this(sparkContext.sc) +/** + * Create new H2OContext based on provided H2O configuration + * + * @param sparkContext Spark Context + * @param conf H2O configuration + */ +class H2OContext private (@transient val sparkContext: SparkContext, @transient conf: H2OConf) extends org.apache.spark.Logging + with Serializable with SparkDataFrameConverter with SupportedRDDConverter with H2OContextUtils{ + self => - /** Runtime list of active H2O nodes */ - private val h2oNodes = mutable.ArrayBuffer.empty[NodeDesc] /** IP of H2O client */ private var localClientIp: String = _ /** REST port of H2O client */ private var localClientPort: Int = _ + /** Runtime list of active H2O nodes */ + private val h2oNodes = mutable.ArrayBuffer.empty[NodeDesc] + + + /** Used backend */ + @transient private val backend: SparklingBackend = new InternalH2OBackend(this) + + + // Check Spark and H2O environment for general arguments independent on backend used and + // also with regards to used backend and store the fix the state of prepared configuration + // so it can't be changed anymore + /** H2O and Spark configuration */ + @transient val _conf = backend.checkAndUpdateConf(conf).clone() + + /** + * This method connects to external H2O cluster if spark.ext.h2o.externalClusterMode is set to true, + * otherwise it creates new H2O cluster living in Spark + */ + def init(): H2OContext = { + if(!isRunningOnCorrectSpark(sparkContext)){ + throw new WrongSparkVersion(s"You are trying to use Sparkling Water built for Spark $buildSparkMajorVersion," + + s" but your $$SPARK_HOME(=${sparkContext.getSparkHome().getOrElse("SPARK_HOME is not defined!")}) property" + + s" points to Spark of version ${sparkContext.version}. Please ensure correct Spark is provided and" + + s" re-run Sparkling Water.") + } + + // Init the H2O Context in a way provided by used backend and return the list of H2O nodes in case of external + // backend or list of spark executors on which H2O runs in case of internal backend + val nodes = backend.init() + + // Fill information about H2O client and H2O nodes in the cluster + h2oNodes.append(nodes:_*) + localClientIp = H2O.SELF_ADDRESS.getHostAddress + localClientPort = H2O.API_PORT + logInfo("Sparkling Water started, status of context: " + this.toString) + + // Store this instance so it can be obtained using getOrCreate method + H2OContext.setInstantiatedContext(this) + this + } + + /** + * Return a copy of this H2OContext's configuration. The configuration ''cannot'' be changed at runtime. + */ + def getConf: H2OConf = _conf.clone() /** Transforms RDD[Supported type] to H2OFrame */ def asH2OFrame(rdd: SupportedRDD): H2OFrame = asH2OFrame(rdd, None) - def asH2OFrame(rdd: SupportedRDD, frameName: Option[String]): H2OFrame = H2OContext.toH2OFrame(sparkContext, rdd, frameName) + def asH2OFrame(rdd: SupportedRDD, frameName: Option[String]): H2OFrame = toH2OFrame(this, rdd, frameName) def asH2OFrame(rdd: SupportedRDD, frameName: String): H2OFrame = asH2OFrame(rdd, Option(frameName)) + /** Transforms RDD[Supported type] to H2OFrame key */ def toH2OFrameKey(rdd: SupportedRDD): Key[_] = toH2OFrameKey(rdd, None) def toH2OFrameKey(rdd: SupportedRDD, frameName: Option[String]): Key[_] = asH2OFrame(rdd, frameName)._key @@ -80,7 +125,7 @@ class H2OContext (@transient val sparkContext: SparkContext) extends { /** Transform DataFrame to H2OFrame */ def asH2OFrame(df : DataFrame): H2OFrame = asH2OFrame(df, None) - def asH2OFrame(df : DataFrame, frameName: Option[String]) : H2OFrame = H2OContext.toH2OFrame(sparkContext, df, if (frameName != null) frameName else None) + def asH2OFrame(df : DataFrame, frameName: Option[String]) : H2OFrame = toH2OFrame(this, df, frameName) def asH2OFrame(df : DataFrame, frameName: String) : H2OFrame = asH2OFrame(df, Option(frameName)) /** Transform DataFrame to H2OFrame key */ @@ -93,54 +138,32 @@ class H2OContext (@transient val sparkContext: SparkContext) extends { /** Create a new H2OFrame based on existing Frame */ def asH2OFrame(fr: Frame): H2OFrame = new H2OFrame(fr) - /** - * Support for calls from Py4J - */ - - /** Conversion from RDD[String] to H2O's DataFrame */ - def asH2OFrameFromRDDString(rdd: JavaRDD[String], frameName: String): H2OFrame = H2OContext.toH2OFrameFromRDDString(sparkContext,rdd.rdd, Option(frameName)) - - /** Returns key of the H2O's DataFrame conversed from RDD[String]*/ - def asH2OFrameFromRDDStringKey(rdd: JavaRDD[String], frameName: String): Key[Frame] = asH2OFrameFromRDDString(rdd, frameName)._key - - /** Conversion from RDD[Boolean] to H2O's DataFrame */ - def asH2OFrameFromRDDBool(rdd: JavaRDD[Boolean], frameName: String): H2OFrame = H2OContext.toH2OFrameFromRDDBool(sparkContext,rdd.rdd, Option(frameName)) - - /** Returns key of the H2O's DataFrame conversed from RDD[Boolean]*/ - def asH2OFrameFromRDDBoolKey(rdd: JavaRDD[Boolean], frameName: String): Key[Frame] = asH2OFrameFromRDDBool(rdd, frameName)._key - - /** Conversion from RDD[Double] to H2O's DataFrame */ - def asH2OFrameFromRDDDouble(rdd: JavaRDD[Double], frameName: String): H2OFrame = H2OContext.toH2OFrameFromRDDDouble(sparkContext,rdd.rdd, Option(frameName)) - /** Returns key of the H2O's DataFrame conversed from RDD[Double]*/ - def asH2OFrameFromRDDDoubleKey(rdd: JavaRDD[Double], frameName: String): Key[Frame] = asH2OFrameFromRDDDouble(rdd, frameName)._key - - /** Conversion from RDD[Long] to H2O's DataFrame */ - def asH2OFrameFromRDDLong(rdd: JavaRDD[Long], frameName: String): H2OFrame = H2OContext.toH2OFrameFromRDDLong(sparkContext,rdd.rdd, Option(frameName)) - - /** Returns key of the H2O's DataFrame conversed from RDD[Long]*/ - def asH2OFrameFromRDDLongKey(rdd: JavaRDD[Long], frameName: String): Key[Frame] = asH2OFrameFromRDDLong(rdd, frameName)._key - - /** Convert given H2O frame into a RDD type */ - @deprecated("Use asRDD instead", "0.2.3") - def toRDD[A <: Product: TypeTag: ClassTag](fr : H2OFrame) : RDD[A] = asRDD[A](fr) - - /** Convert given H2O frame into a Product RDD type */ - def asRDD[A <: Product: TypeTag: ClassTag](fr : H2OFrame) : RDD[A] = createH2ORDD[A, H2OFrame](fr) + /** Convert given H2O frame into a Product RDD type + * + * Consider using asH2OFrame since asRDD has several limitations such as that asRDD can't be used in Spark REPL + * in case we are RDD[T] where T is class defined in REPL. This is because class T is created as inner class + * and we are not able to create instance of class T without outer scope - which is impossible to get. + * */ + def asRDD[A <: Product: TypeTag: ClassTag](fr : H2OFrame) : RDD[A] = toRDD[A, H2OFrame](this, fr) /** A generic convert of Frame into Product RDD type * + * Consider using asH2OFrame since asRDD has several limitations such as that asRDD can't be used in Spark REPL + * in case we are RDD[T] where T is class defined in REPL. This is because class T is created as inner class + * and we are not able to create instance of class T without outer scope - which is impossible to get. + * * This code: hc.asRDD[PUBDEV458Type](rdd) will need to be call as hc.asRDD[PUBDEV458Type].apply(rdd) */ def asRDD[A <: Product : TypeTag : ClassTag] = new { - def apply[T <: Frame](fr: T): RDD[A] = createH2ORDD[A, T](fr) + def apply[T <: Frame](fr: T): RDD[A] = toRDD[A, T](H2OContext.this, fr) } /** Convert given H2O frame into DataFrame type */ @deprecated("1.3", "Use asDataFrame") - def asSchemaRDD[T <: Frame](fr : T, copyMetadata: Boolean = true)(implicit sqlContext: SQLContext) : DataFrame = createH2OSchemaRDD(fr, copyMetadata) - def asDataFrame[T <: Frame](fr : T, copyMetadata: Boolean = true)(implicit sqlContext: SQLContext) : DataFrame = createH2OSchemaRDD(fr, copyMetadata) - def asDataFrame(s : String, copyMetadata: Boolean)(implicit sqlContext: SQLContext) : DataFrame = createH2OSchemaRDD(new H2OFrame(s), copyMetadata) + def asSchemaRDD[T <: Frame](fr : T, copyMetadata: Boolean = true)(implicit sqlContext: SQLContext) : DataFrame = toDataFrame(this, fr, copyMetadata) + def asDataFrame[T <: Frame](fr : T, copyMetadata: Boolean = true)(implicit sqlContext: SQLContext) : DataFrame = toDataFrame(this, fr, copyMetadata) + def asDataFrame(s : String, copyMetadata: Boolean)(implicit sqlContext: SQLContext) : DataFrame = toDataFrame(this, new H2OFrame(s), copyMetadata) def h2oLocalClient = this.localClientIp + ":" + this.localClientPort @@ -151,180 +174,33 @@ class H2OContext (@transient val sparkContext: SparkContext) extends { // For now disable opening Spark UI //def sparkUI = sparkContext.ui.map(ui => ui.appUIAddress) - /** Initialize Sparkling H2O and start H2O cloud with specified number of workers. */ - @deprecated(message = "Use start() method.", since = "1.5.11") - def start(h2oWorkers: Int):H2OContext = { - import H2OConf._ - sparkConf.set(PROP_CLUSTER_SIZE._1, h2oWorkers.toString) - start() - - } - - /** Initialize Sparkling H2O and start H2O cloud. */ - def start(): H2OContext = { - if(!isRunningOnCorrectSpark){ - throw new WrongSparkVersion(s"You are trying to use Sparkling Water built for Spark ${buildSparkMajorVersion}," + - s" but your $$SPARK_HOME(=${sparkContext.getSparkHome().getOrElse("SPARK_HOME is not defined!")}) property" + - s" points to Spark of version ${sparkContext.version}. Please ensure correct Spark is provided and" + - s" re-run Sparkling Water.") - } - import H2OConf._ - // Setup properties for H2O configuration - if (!sparkConf.contains(PROP_CLOUD_NAME._1)) { - sparkConf.set(PROP_CLOUD_NAME._1, - PROP_CLOUD_NAME._2 + System.getProperty("user.name", "cluster") + "_" + Random.nextInt()) - } - - // Check Spark environment and reconfigure some values (Note: this is useless in more of the cases since SparkContext is already running) - H2OContext.checkAndUpdateSparkEnv(sparkContext, sparkConf, isFailOnUnsupportedSparkParamEnabled) - logInfo(s"Starting H2O services: " + super[H2OConf].toString) - // Create dummy RDD distributed over executors - - val (spreadRDD, spreadRDDNodes) = createSpreadRDD() - - if(isClusterTopologyListenerEnabled){ - //attach listener which shutdown H2O when we bump into executor we didn't discover during the spreadRDD phase - sparkContext.addSparkListener(new SparkListener(){ - override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = { - throw new IllegalArgumentException("Executor without H2O instance discovered, killing the cloud!") - } - }) - } - // Start H2O nodes - // Get executors to execute H2O - val allExecutorIds = spreadRDDNodes.map(_.executorId).distinct - val executorIds = allExecutorIds - // The collected executors based on IDs should match - assert(spreadRDDNodes.length == executorIds.length, - s"Unexpected number of executors ${spreadRDDNodes.length}!=${executorIds.length}") - // H2O is executed only on the subset of Spark cluster - fail - if (executorIds.length < allExecutorIds.length) { - throw new IllegalArgumentException(s"""Spark cluster contains ${allExecutorIds.length}, - but H2O is running only on ${executorIds.length} nodes!""") - } - // Execute H2O on given nodes - logInfo(s"""Launching H2O on following ${spreadRDDNodes.length} nodes: ${spreadRDDNodes.mkString(",")}""") - - var h2oNodeArgs = getH2ONodeArgs - // Disable web on h2o nodes in non-local mode - if(!sparkContext.isLocal){ - h2oNodeArgs = h2oNodeArgs ++ Array("-disable_web") - } - logDebug(s"Arguments used for launching h2o nodes: ${h2oNodeArgs.mkString(" ")}") - val executors = startH2O(sparkContext, spreadRDD, spreadRDDNodes.length, h2oNodeArgs, nodeNetworkMask) - // Store runtime information - h2oNodes.append( executors:_* ) - - // Connect to a cluster via H2O client, but only in non-local case - if (!sparkContext.isLocal) { - logTrace("Sparkling H2O - DISTRIBUTED mode: Waiting for " + executors.length) - // Do not use IP if network mask is specified - val h2oClientIp = clientIp.getOrElse(getHostname(SparkEnv.get)) - // Get arguments for this launch including flatfile - val h2oClientArgs = toH2OArgs( - getH2OClientArgs(h2oClientIp), - this, - executors) - logDebug(s"Arguments used for launching h2o client node: ${h2oClientArgs.mkString(" ")}") - // Launch H2O - H2OStarter.start(h2oClientArgs, false) - } - // And wait for right cluster size - H2O.waitForCloudSize(executors.length, cloudTimeout) - // Register web API for client - H2OContext.registerClientWebAPI(sparkContext, this) - H2O.finalizeRegistration() - // Fill information about H2O client - localClientIp = H2O.SELF_ADDRESS.getHostAddress - localClientPort = H2O.API_PORT - - // Inform user about status - logInfo("Sparkling Water started, status of context: " + this.toString) - this - } - /** Stops H2O context. * - * Calls System.exit() which kills executor JVM. + * @param stopSparkContext stop also spark context */ - def stop(stopSparkContext: Boolean = false): Unit = { - if (stopSparkContext) sparkContext.stop() - H2O.orderlyShutdown(1000) - H2O.exit(0) - } - - private def createSpreadRDD() = new SpreadRDDBuilder(sparkContext, - H2OContextUtils.guessTotalExecutorSize(sparkContext)).build() - - - def createH2ORDD[A <: Product: TypeTag: ClassTag, T <: Frame](fr: T): RDD[A] = { - new H2ORDD[A, T](fr)(sparkContext) - } - - /** - * Create a Spark DataFrame from given H2O frame. - * - * @param fr an instnace of H2O frame - * @param copyMetadata copy H2O metadata into Spark DataFrame - * @param sqlContext running sqlContext - * @tparam T type of H2O frame - * @return a new DataFrame - */ - def createH2OSchemaRDD[T <: Frame](fr: T, copyMetadata: Boolean)(implicit sqlContext: SQLContext): DataFrame = { - val ss = new H2OFrameRelation(fr, copyMetadata)(sqlContext) - sqlContext.baseRelationToDataFrame(ss) - } + def stop(stopSparkContext: Boolean = false): Unit = backend.stop(stopSparkContext) /** Open H2O Flow running in this client. */ - def openFlow(): Unit = openURI(s"http://${h2oLocalClient}") + def openFlow(): Unit = openURI(sparkContext, s"http://$h2oLocalClient") + /** Open Spark task manager. */ //def openSparkUI(): Unit = sparkUI.foreach(openURI(_)) - /** Open browser for given address. - * - * @param uri addres to open in browser, e.g., http://example.com - */ - private def openURI(uri: String): Unit = { - import java.awt.Desktop - if (!isTesting) { - if (Desktop.isDesktopSupported) { - Desktop.getDesktop.browse(new java.net.URI(uri)) - } else { - logWarning(s"Desktop support is missing! Cannot open browser for ${uri}") - } - } - } - - /** - * Return true if running inside spark/sparkling water test. - * - * @return true if the actual run is test run - * @return true if the actual run is test run - */ - private def isTesting = sparkContext.conf.contains("spark.testing") || sys.props.contains("spark.testing") - override def toString: String = { s""" |Sparkling Water Context: | * H2O name: ${H2O.ARGS.name} - | * number of executors: ${h2oNodes.size} - | * list of used executors: + | * cluster size: ${h2oNodes.size} + | * list of used nodes: | (executorId, host, port) | ------------------------ | ${h2oNodes.mkString("\n ")} | ------------------------ | - | Open H2O Flow in browser: http://${h2oLocalClient} (CMD + click in Mac OSX) + | Open H2O Flow in browser: http://$h2oLocalClient (CMD + click in Mac OSX) """.stripMargin } - /** Checks whether version of provided Spark is the same as Spark's version designated for this Sparkling Water version. - * We check for correct version in shell scripts and during the build but we need to do the check also in the code in cases when the user - * executes for example spark-shell command with sparkling water assembly jar passed as --jars and initiates H2OContext. - * (Because in that case no check for correct Spark version has been done so far.) - */ - private def isRunningOnCorrectSpark = sparkContext.version.startsWith(buildSparkMajorVersion) - // scalastyle:off // Disable style checker so "implicits" object can start with lowercase i /** Define implicits available via h2oContext.implicits._*/ @@ -332,14 +208,9 @@ class H2OContext (@transient val sparkContext: SparkContext) extends { protected override def _h2oContext: H2OContext = self } // scalastyle:on - H2OContext.setInstantiatedContext(this) } -object H2OContext extends Logging { - - val UNSUPPORTED_SPARK_OPTIONS = Seq( - ("spark.dynamicAllocation.enabled", "true"), - ("spark.speculation", "true")) +object H2OContext extends Logging{ private[H2OContext] def setInstantiatedContext(h2oContext: H2OContext): Unit = { synchronized { @@ -360,541 +231,34 @@ object H2OContext extends Logging { */ def get(): Option[H2OContext] = Option(instantiatedContext.get()) - private def getOrCreate(sc: SparkContext, h2oWorkers: Option[Int]): H2OContext = synchronized { - if (instantiatedContext.get() == null) { - instantiatedContext.set(new H2OContext(sc)) - if(h2oWorkers.isEmpty){ - instantiatedContext.get().start() - }else{ - instantiatedContext.get().start(h2oWorkers.get) - } - } - instantiatedContext.get() - } - /** - * Get existing H2O Context or initialize Sparkling H2O and start H2O cloud with specified number of workers + * Get existing or create new H2OContext based on provided H2O configuration * * @param sc Spark Context + * @param conf H2O configuration * @return H2O Context */ - @deprecated(message = "Use getOrCreate(sc: SparkContext) method.", since = "1.5.11") - def getOrCreate(sc: SparkContext, h2oWorkers: Int): H2OContext = { - getOrCreate(sc, Some(h2oWorkers)) + def getOrCreate(sc: SparkContext, conf: H2OConf): H2OContext = synchronized { + if (instantiatedContext.get() == null) { + instantiatedContext.set(new H2OContext(sc, conf)) + instantiatedContext.get().init() + } + instantiatedContext.get() } /** - * Get existing H2O Context or initialize Sparkling H2O and start H2O cloud with default number of workers + * Get existing or create new H2OContext based on provided H2O configuration. It searches the configuration + * properties passed to Sparkling Water and based on them starts H2O Context. If the values are not found, the default + * values are used in most of the cases. The default cluster mode is internal, ie. spark.ext.h2o.external.cluster.mode=false * * @param sc Spark Context * @return H2O Context */ def getOrCreate(sc: SparkContext): H2OContext = { - getOrCreate(sc, None) - } - - /** Supports call from java environments. */ - def getOrCreate(sc: JavaSparkContext): H2OContext = { - getOrCreate(sc.sc, None) - } - - /** Supports call from java environments. */ - @deprecated(message = "Use getOrCreate(sc: JavaSparkContext) method.", since = "1.5.11") - def getOrCreate(sc: JavaSparkContext, h2oWorkers: Int): H2OContext = { - getOrCreate(sc.sc,Some(h2oWorkers)) - } - - /** Transform SchemaRDD into H2O Frame */ - def toH2OFrame(sc: SparkContext, dataFrame: DataFrame, frameKeyName: Option[String]) : H2OFrame = { - import org.apache.spark.h2o.H2OSchemaUtils._ - // Cache DataFrame RDD's - val dfRdd = dataFrame.rdd - - val keyName = frameKeyName.getOrElse("frame_rdd_" + dfRdd.id) - // Fetch cached frame from DKV - val frameVal = DKV.get(keyName) - if (frameVal==null) { - // Flattens and expands RDD's schema - val flatRddSchema = expandedSchema(sc, dataFrame) - // Patch the flat schema based on information about types - val fnames = flatRddSchema.map(t => t._2.name).toArray - // Transform datatype into h2o types - val vecTypes = flatRddSchema.indices - .map(idx => { - val f = flatRddSchema(idx) - dataTypeToVecType(f._2.dataType) - }).toArray - // Prepare frame header and put it into DKV under given name - initFrame(keyName, fnames) - // Create a new chunks corresponding to spark partitions - // Note: Eager, not lazy, evaluation - val rows = sc.runJob(dfRdd, perSQLPartition(keyName, flatRddSchema, vecTypes) _) - val res = new Array[Long](dfRdd.partitions.size) - rows.foreach { case (cidx, nrows) => res(cidx) = nrows} - - // Add Vec headers per-Chunk, and finalize the H2O Frame - new H2OFrame( - finalizeFrame( - keyName, - res, - vecTypes)) - } else { - new H2OFrame(frameVal.get.asInstanceOf[Frame]) - } - } - - private def inferFieldType(value : Any): Class[_] ={ - value match { - case n: Byte => classOf[java.lang.Byte] - case n: Short => classOf[java.lang.Short] - case n: Int => classOf[java.lang.Integer] - case n: Long => classOf[java.lang.Long] - case n: Float => classOf[java.lang.Float] - case n: Double => classOf[java.lang.Double] - case n: Boolean => classOf[java.lang.Boolean] - case n: String => classOf[java.lang.String] - case n: java.sql.Timestamp => classOf[java.sql.Timestamp] - case q => throw new IllegalArgumentException(s"Do not understand type $q") - } - } - - def toH2OFrameFromPureProduct(sc: SparkContext, rdd: RDD[Product], frameKeyName: Option[String]): H2OFrame = { - val keyName = frameKeyName.getOrElse("frame_rdd_" + rdd.id + Key.rand()) // There are uniq IDs for RDD - - // infer the type - val first = rdd.first() - val fnames = 0.until(first.productArity).map(idx => "f" + idx).toArray[String] - val ftypes = new ListBuffer[Class[_]]() - val it = first.productIterator - while(it.hasNext){ - ftypes+=inferFieldType(it.next()) - } - // Collect H2O vector types for all input types - val vecTypes = ftypes.toArray[Class[_]].indices.map(idx => dataTypeToVecType(ftypes(idx))).toArray - // Make an H2O data Frame - but with no backing data (yet) - initFrame(keyName, fnames) - // Create chunks on remote nodes - val rows = sc.runJob(rdd, perTypedRDDPartition(keyName, vecTypes) _) // eager, not lazy, evaluation - val res = new Array[Long](rdd.partitions.length) - rows.foreach{ case(cidx, nrows) => res(cidx) = nrows } - - // Add Vec headers per-Chunk, and finalize the H2O Frame - new H2OFrame(finalizeFrame(keyName, res, vecTypes)) - } - /** Transform typed RDD into H2O Frame */ - def toH2OFrame[A <: Product : TypeTag](sc: SparkContext, rdd: RDD[A], frameKeyName: Option[String]) : H2OFrame = { - import org.apache.spark.h2o.H2OTypeUtils._ - import org.apache.spark.h2o.ReflectionUtils._ - - val keyName = frameKeyName.getOrElse("frame_rdd_" + rdd.id + Key.rand()) // There are uniq IDs for RDD - val fnames = names[A] - val ftypes = types[A](fnames) - // Collect H2O vector types for all input types - val vecTypes = ftypes.indices.map(idx => dataTypeToVecType(ftypes(idx))).toArray - // Make an H2O data Frame - but with no backing data (yet) - initFrame(keyName, fnames) - // Create chunks on remote nodes - val rows = sc.runJob(rdd, perTypedRDDPartition(keyName, vecTypes) _) // eager, not lazy, evaluation - val res = new Array[Long](rdd.partitions.length) - rows.foreach{ case(cidx, nrows) => res(cidx) = nrows } - - // Add Vec headers per-Chunk, and finalize the H2O Frame - new H2OFrame(finalizeFrame(keyName, res, vecTypes)) - } - - /** Transform supported type for conversion to H2OFrame*/ - def toH2OFrame(sc: SparkContext, rdd: SupportedRDD, frameKeyName: Option[String]): H2OFrame = rdd.toH2OFrame(sc, frameKeyName) - - /** Transform RDD[String] to appropriate H2OFrame */ - def toH2OFrameFromRDDString(sc: SparkContext, rdd: RDD[String], frameKeyName: Option[String]): H2OFrame = toH2OFrameFromPrimitiveRDD(sc, rdd, frameKeyName) - - /** Transform RDD[Int] to appropriate H2OFrame */ - def toH2OFrameFromRDDInt(sc: SparkContext, rdd: RDD[Int], frameKeyName: Option[String]): H2OFrame = toH2OFrameFromPrimitiveRDD(sc, rdd, frameKeyName) - - /** Transform RDD[Float] to appropriate H2OFrame */ - def toH2OFrameFromRDDFloat(sc: SparkContext, rdd: RDD[Float], frameKeyName: Option[String]): H2OFrame = toH2OFrameFromPrimitiveRDD(sc, rdd, frameKeyName) - - /** Transform RDD[Double] to appropriate H2OFrame */ - def toH2OFrameFromRDDDouble(sc: SparkContext, rdd: RDD[Double], frameKeyName: Option[String]): H2OFrame = toH2OFrameFromPrimitiveRDD(sc, rdd, frameKeyName) - - /** Transform RDD[Long] to appropriate H2OFrame */ - def toH2OFrameFromRDDLong(sc: SparkContext, rdd: RDD[Long], frameKeyName: Option[String]): H2OFrame = toH2OFrameFromPrimitiveRDD(sc, rdd, frameKeyName) - - /** Transform RDD[Double] to appropriate H2OFrame */ - def toH2OFrameFromRDDBool(sc: SparkContext, rdd: RDD[Boolean], frameKeyName: Option[String]): H2OFrame = toH2OFrameFromPrimitiveRDD(sc, rdd, frameKeyName) - - /** Transform RDD[Short] to appropriate H2OFrame */ - def toH2OFrameFromRDDShort(sc: SparkContext, rdd: RDD[Short], frameKeyName: Option[String]): H2OFrame = toH2OFrameFromPrimitiveRDD(sc, rdd, frameKeyName) - - /** Transform RDD[Byte] to appropriate H2OFrame */ - def toH2OFrameFromRDDByte(sc: SparkContext, rdd: RDD[Byte], frameKeyName: Option[String]): H2OFrame = toH2OFrameFromPrimitiveRDD(sc, rdd, frameKeyName) - - /** Transform RDD[Byte] to appropriate H2OFrame */ - def toH2OFrameFromRDDTimeStamp(sc: SparkContext, rdd: RDD[java.sql.Timestamp], frameKeyName: Option[String]): H2OFrame = toH2OFrameFromPrimitiveRDD(sc, rdd, frameKeyName) // scalastyle:ignore - - private[this] - def toH2OFrameFromPrimitiveRDD[T: TypeTag](sc: SparkContext, rdd: RDD[T], frameKeyName: Option[String]): H2OFrame = { - import org.apache.spark.h2o.H2OTypeUtils._ - import org.apache.spark.h2o.ReflectionUtils._ - - val keyName = frameKeyName.getOrElse("frame_rdd_" + rdd.id + Key.rand()) - - val fnames = Array[String]("values") - val ftypes = Array[Class[_]](typ(typeOf[T])) - val vecTypes = ftypes.indices.map(idx => dataTypeToVecType(ftypes(idx))).toArray - - // Make an H2O data Frame - but with no backing data (yet) - initFrame(keyName, fnames) - - val rows = sc.runJob(rdd, perPrimitiveRDDPartition(keyName, vecTypes) _) // eager, not lazy, evaluation - val res = new Array[Long](rdd.partitions.length) - rows.foreach { case (cidx, nrows) => res(cidx) = nrows } - - // Add Vec headers per-Chunk, and finalize the H2O Frame - new H2OFrame(finalizeFrame(keyName, res, vecTypes)) - } - - /** Transform RDD[LabeledPoint] to appropriate H2OFrame */ - def toH2OFrameFromRDDLabeledPoint(sc: SparkContext, rdd: RDD[LabeledPoint], frameKeyName: Option[String]): H2OFrame = { - import org.apache.spark.h2o.H2OTypeUtils._ - import org.apache.spark.h2o.ReflectionUtils._ - // first convert vector to dense vector - val rddDense = rdd.map(labeledPoint => new LabeledPoint(labeledPoint.label,labeledPoint.features.toDense)) - val numFeatures = rddDense.map(labeledPoint => labeledPoint.features.size) - val maxNumFeatures = numFeatures.max() - val minNumFeatures = numFeatures.min() - if(minNumFeatures "feature" + num).toSeq).toArray[String] - val ftypes = 0.until(maxNumFeatures + 1).map(_ => typ(typeOf[Double])) - val vecTypes = ftypes.indices.map(idx => dataTypeToVecType(ftypes(idx))).toArray - // Make an H2O data Frame - but with no backing data (yet) - initFrame(keyName, fnames) - val rows = sc.runJob(rddDense, perLabeledPointRDDPartition(keyName, vecTypes, maxNumFeatures) _) // eager, not lazy, evaluation - val res = new Array[Long](rdd.partitions.length) - rows.foreach { case (cidx, nrows) => res(cidx) = nrows } - - // Add Vec headers per-Chunk, and finalize the H2O Frame - new H2OFrame(finalizeFrame(keyName, res, vecTypes)) - } - - private - def perPrimitiveRDDPartition[T](keystr: String, vecTypes: Array[Byte]) - (context: TaskContext, it: Iterator[T]): (Int, Long) = { - // An array of H2O NewChunks; A place to record all the data in this partition - val nchks = water.fvec.FrameUtils.createNewChunks(keystr, vecTypes, context.partitionId) - // Helper to hold H2O string - val valStr = new BufferedString() - it.foreach(r => { - // For all rows in RDD - val chk = nchks(0) // There is only one chunk - r match { - case n: Number => chk.addNum(n.doubleValue()) - case n: Boolean => chk.addNum(if (n) 1 else 0) - case n: String => chk.addStr(valStr.set(n)) - case n : java.sql.Timestamp => chk.addNum(n.asInstanceOf[java.sql.Timestamp].getTime()) - case _ => chk.addNA() - } - }) - // Compress & write out the Partition/Chunks - water.fvec.FrameUtils.closeNewChunks(nchks) - // Return Partition# and rows in this Partition - (context.partitionId, nchks(0)._len) - } - private - def perSQLPartition ( keystr: String, types: Seq[(Seq[Int], StructField, Byte)], vecTypes: Array[Byte]) - ( context: TaskContext, it: Iterator[Row] ): (Int,Long) = { - // New chunks on remote node - val nchks = water.fvec.FrameUtils.createNewChunks(keystr, vecTypes, context.partitionId) - val valStr = new BufferedString() // just helper for string columns - it.foreach(row => { - var startOfSeq = -1 - // Fill row in the output frame - types.indices.foreach { idx => // Index of column - val chk = nchks(idx) - val field = types(idx) - val path = field._1 - val dataType = field._2.dataType - // Helpers to distinguish embedded collection types - val isAry = field._3 == H2OSchemaUtils.ARRAY_TYPE - val isVec = field._3 == H2OSchemaUtils.VEC_TYPE - val isNewPath = if (idx > 0) path != types(idx-1)._1 else true - // Reset counter for sequences - if ((isAry || isVec) && isNewPath) startOfSeq = idx - else if (!isAry && !isVec) startOfSeq = -1 - - var i = 0 - var subRow = row - while (i < path.length-1 && !subRow.isNullAt(path(i))) { - subRow = subRow.getAs[Row](path(i)); i += 1 - } - val aidx = path(i) // actual index into row provided by path - if (subRow.isNullAt(aidx)) { - chk.addNA() - } else { - val ary = if (isAry) subRow.getAs[Seq[_]](aidx) else null - val aryLen = if (isAry) ary.length else -1 - val aryIdx = idx - startOfSeq // shared index to position in array/vector - val vec = if (isVec) subRow.getAs[mllib.linalg.Vector](aidx) else null - if (isAry && aryIdx >= aryLen) chk.addNA() - else if (isVec && aryIdx >= vec.size) chk.addNum(0.0) // Add zeros for vectors - else dataType match { - case BooleanType => chk.addNum(if (isAry) - if (ary(aryIdx).asInstanceOf[Boolean]) 1 else 0 - else if (subRow.getBoolean(aidx)) 1 else 0) - case BinaryType => - case ByteType => chk.addNum(if (isAry) ary(aryIdx).asInstanceOf[Byte] else subRow.getByte(aidx)) - case ShortType => chk.addNum(if (isAry) ary(aryIdx).asInstanceOf[Short] else subRow.getShort(aidx)) - case IntegerType => chk.addNum(if (isAry) ary(aryIdx).asInstanceOf[Int] else subRow.getInt(aidx)) - case LongType => chk.addNum(if (isAry) ary(aryIdx).asInstanceOf[Long] else subRow.getLong(aidx)) - case FloatType => chk.addNum(if (isAry) ary(aryIdx).asInstanceOf[Float] else subRow.getFloat(aidx)) - case DoubleType => chk.addNum(if (isAry) { - ary(aryIdx).asInstanceOf[Double] - } else { - if (isVec) { - subRow.getAs[mllib.linalg.Vector](aidx)(idx - startOfSeq) - } else { - subRow.getDouble(aidx) - } - }) - case StringType => { - val sv = if (isAry) ary(aryIdx).asInstanceOf[String] else subRow.getString(aidx) - // Always produce string vectors - chk.addStr(valStr.set(sv)) - } - case TimestampType => chk.addNum(subRow.getAs[java.sql.Timestamp](aidx).getTime()) - case _ => chk.addNA() - } - } - - } - }) - // Compress & write out the Partition/Chunks - water.fvec.FrameUtils.closeNewChunks(nchks) - // Return Partition# and rows in this Partition - (context.partitionId,nchks(0)._len) - } - - private - def perTypedRDDPartition[A<:Product](keystr:String, vecTypes: Array[Byte]) - ( context: TaskContext, it: Iterator[A] ): (Int,Long) = { - // An array of H2O NewChunks; A place to record all the data in this partition - val nchks = water.fvec.FrameUtils.createNewChunks(keystr, vecTypes, context.partitionId) - - val valStr = new BufferedString() - it.foreach(prod => { // For all rows which are subtype of Product - for( i <- 0 until prod.productArity ) { // For all fields... - val fld = prod.productElement(i) - val chk = nchks(i) - val x = fld match { - case Some(n) => n - case _ => fld - } - x match { - case n: Number => chk.addNum(n.doubleValue()) - case n: Boolean => chk.addNum(if (n) 1 else 0) - case n: String => chk.addStr(valStr.set(n)) - case n : java.sql.Timestamp => chk.addNum(n.asInstanceOf[java.sql.Timestamp].getTime()) - case _ => chk.addNA() - } - } - }) - // Compress & write out the Partition/Chunks - water.fvec.FrameUtils.closeNewChunks(nchks) - // Return Partition# and rows in this Partition - (context.partitionId,nchks(0)._len) - } - - private - def perLabeledPointRDDPartition(keystr: String, vecTypes: Array[Byte], maxNumFeatures: Int) - (context: TaskContext, it: Iterator[LabeledPoint]): (Int, Long) = { - // An array of H2O NewChunks; A place to record all the data in this partition - val chunks = water.fvec.FrameUtils.createNewChunks(keystr, vecTypes, context.partitionId) - it.foreach(labeledPoint => { - // For all LabeledPoints in RDD - var nextChunkId = 0 - - // Add LabeledPoint label - chunks(nextChunkId).addNum(labeledPoint.label) - nextChunkId = nextChunkId + 1 - - for( i<-0 until labeledPoint.features.size) { - // For all features... - chunks(nextChunkId).addNum(labeledPoint.features(i)) - nextChunkId =nextChunkId + 1 - } - - for( i<-labeledPoint.features.size until maxNumFeatures){ - // Fill missing features with n/a - chunks(nextChunkId).addNA() - nextChunkId = nextChunkId + 1 - } - }) - // Compress & write out the Partition/Chunks - water.fvec.FrameUtils.closeNewChunks(chunks) - // Return Partition# and rows in this Partition - (context.partitionId, chunks(0)._len) - } - - private - def initFrame[T](keyName: String, names: Array[String]):Unit = { - val fr = new water.fvec.Frame(Key.make(keyName)) - water.fvec.FrameUtils.preparePartialFrame(fr, names) - // Save it directly to DKV - fr.update() - } - - private - def finalizeFrame[T](keyName: String, - res: Array[Long], - colTypes: Array[Byte], - colDomains: Array[Array[String]] = null):Frame = { - val fr:Frame = DKV.get(keyName).get.asInstanceOf[Frame] - water.fvec.FrameUtils.finalizePartialFrame(fr, res, colDomains, colTypes) - fr - } - - /** Check Spark environment and warn about possible problems. */ - private - def checkAndUpdateSparkEnv(sparkContext: SparkContext, conf: SparkConf, exitOnUnsupportedParam: Boolean): Unit = { - // If 'spark.executor.instances' is specified update H2O property as well - conf.getOption("spark.executor.instances").foreach(v => conf.set("spark.ext.h2o.cluster.size", v)) - // Increase locality timeout since h2o-specific tasks can be long computing - if (conf.getInt("spark.locality.wait", 3000) <= 3000 && !sparkContext.isLocal) { - logWarning(s"Increasing 'spark.locality.wait' to value 30000") - conf.set("spark.locality.wait", "30000") - } - - if (!conf.contains("spark.scheduler.minRegisteredResourcesRatio") && !sparkContext.isLocal) { - logWarning("The property 'spark.scheduler.minRegisteredResourcesRatio' is not specified!\n" + - "We recommend to pass `--conf spark.scheduler.minRegisteredResourcesRatio=1`") - // Setup the property but at this point it does not make good sense - conf.set("spark.scheduler.minRegisteredResourcesRatio", "1") - } - - H2OContext.UNSUPPORTED_SPARK_OPTIONS.foreach(opt => if (conf.contains(opt._1) && (opt._2 == None || conf.get(opt._1) == opt._2)) { - logWarning(s"Unsupported options ${opt._1} detected!") - if (exitOnUnsupportedParam) { - logWarning( - s""" - |The application is going down, since the parameter ${H2OConf.PROP_FAIL_ON_UNSUPPORTED_SPARK_PARAM} is true! - |If you would like to skip the fail call, please, specify the value of the parameter to false. - """.stripMargin) - throw new IllegalArgumentException(s"Unsupported argument: ${opt}") - } - }) - } - private[h2o] def registerClientWebAPI(sc: SparkContext, h2oContext: H2OContext): Unit = { - // Workaround for [SW-132] - val doEnableRepl = h2oContext.isH2OReplEnabled && { - try { - classOf[SparkIMain].getDeclaredField("classServer") - true - } catch { - case e: NoSuchFieldException => false - } - } - if(doEnableRepl){ - registerScalaIntEndp(sc) - } - registerDataFramesEndp(sc, h2oContext) - registerH2OFramesEndp(sc, h2oContext) - registerRDDsEndp(sc, h2oContext) + getOrCreate(sc, new H2OConf(sc)) } - private def registerH2OFramesEndp(sc: SparkContext, h2oContext: H2OContext) = { - - val h2oFramesHandler = new H2OFramesHandler(sc, h2oContext) - - def h2oFramesFactory = new HandlerFactory { - override def create(handler: Class[_ <: Handler]): Handler = h2oFramesHandler - } - - RequestServer.registerEndpoint("getDataFrame","POST", "/3/h2oframes/{h2oframe_id}/dataframe", - classOf[H2OFramesHandler], "toDataFrame", - "Transform H2OFrame with given ID to Spark's DataFrame", - h2oFramesFactory) - - } - - private def registerRDDsEndp(sc: SparkContext, h2oContext: H2OContext) = { - - val rddsHandler = new RDDsHandler(sc, h2oContext) - - def rddsFactory = new HandlerFactory { - override def create(aClass: Class[_ <: Handler]): Handler = rddsHandler - } - - RequestServer.registerEndpoint("listRDDs", "GET", "/3/RDDs", - classOf[RDDsHandler], "list", - "Return all RDDs within Spark cloud", - rddsFactory) - - RequestServer.registerEndpoint("getRDD", "POST", "/3/RDDs/{rdd_id}", - classOf[RDDsHandler], "getRDD", - "Get RDD with the given ID from Spark cloud", - rddsFactory) - - RequestServer.registerEndpoint("rddToH2OFrame", "POST", "/3/RDDs/{rdd_id}/h2oframe", - classOf[RDDsHandler], "toH2OFrame", - "Transform RDD with the given ID to H2OFrame", - rddsFactory) - - } - - private def registerDataFramesEndp(sc: SparkContext, h2oContext: H2OContext) = { - - val dataFramesHandler = new DataFramesHandler(sc, h2oContext) - - def dataFramesfactory = new HandlerFactory { - override def create(aClass: Class[_ <: Handler]): Handler = dataFramesHandler - } - - RequestServer.registerEndpoint("listDataFrames", "GET", "/3/dataframes", - classOf[DataFramesHandler], "list", - "Return all Spark's DataFrames", - dataFramesfactory) - - RequestServer.registerEndpoint("getDataFrame", "POST", "/3/dataframes/{dataframe_id}", - classOf[DataFramesHandler], "getDataFrame", - "Get Spark's DataFrame with the given ID", - dataFramesfactory) - - RequestServer.registerEndpoint("dataFrametoH2OFrame", "POST", "/3/dataframes/{dataframe_id}/h2oframe", - classOf[DataFramesHandler], "toH2OFrame", - "Transform Spark's DataFrame with the given ID to H2OFrame", - dataFramesfactory) - - } - - private def registerScalaIntEndp(sc: SparkContext) = { - val scalaCodeHandler = new ScalaCodeHandler(sc) - def scalaCodeFactory = new HandlerFactory { - override def create(aClass: Class[_ <: Handler]): Handler = scalaCodeHandler - } - RequestServer.registerEndpoint("interpretScalaCode", "POST" ,"/3/scalaint/{session_id}", - classOf[ScalaCodeHandler], "interpret", - "Interpret the code and return the result", - scalaCodeFactory) - - RequestServer.registerEndpoint("initScalaSession", "POST", "/3/scalaint", - classOf[ScalaCodeHandler], "initSession", - "Return session id for communication with scala interpreter", - scalaCodeFactory) - - RequestServer.registerEndpoint("getScalaSessions", "GET" ,"/3/scalaint", - classOf[ScalaCodeHandler], "getSessions", - "Return all active session IDs", - scalaCodeFactory) - - RequestServer.registerEndpoint("destroyScalaSession", "DELETE", "/3/scalaint/{session_id}", - classOf[ScalaCodeHandler], "destroySession", - "Return session id for communication with scala interpreter", - scalaCodeFactory) - } } -class WrongSparkVersion(msg: String) extends Exception(msg) with NoStackTrace {} +class WrongSparkVersion(msg: String) extends Exception(msg) with NoStackTrace + diff --git a/core/src/main/scala/org/apache/spark/h2o/H2OContextImplicits.scala b/core/src/main/scala/org/apache/spark/h2o/H2OContextImplicits.scala index 9056e5a1c..93847b7c4 100644 --- a/core/src/main/scala/org/apache/spark/h2o/H2OContextImplicits.scala +++ b/core/src/main/scala/org/apache/spark/h2o/H2OContextImplicits.scala @@ -23,12 +23,13 @@ import water.Key import scala.reflect.runtime.universe._ /** - * Holder for implicit conversions available on H2OContext + * Implicit transformations available on [[org.apache.spark.h2o.H2OContext]] */ abstract class H2OContextImplicits { + protected def _h2oContext: H2OContext /** Implicit conversion from RDD[Supported type] to H2OFrame */ - implicit def asH2OFrameFromRDDProduct[A <: Product : TypeTag](rdd : RDD[A]): H2OFrame = _h2oContext.asH2OFrame(rdd,None) + implicit def asH2OFrameFromRDDProduct[A <: Product : TypeTag](rdd : RDD[A]): H2OFrame = _h2oContext.asH2OFrame(rdd, None) implicit def asH2OFrameFromRDDString(rdd: RDD[String]): H2OFrame = _h2oContext.asH2OFrame(rdd,None) implicit def asH2OFrameFromRDDBool(rdd: RDD[Boolean]): H2OFrame = _h2oContext.asH2OFrame(rdd,None) implicit def asH2OFrameFromRDDDouble(rdd: RDD[Double]): H2OFrame = _h2oContext.asH2OFrame(rdd,None) @@ -61,6 +62,7 @@ abstract class H2OContextImplicits { /** Implicit conversion from Frame(H2O) to H2OFrame key */ implicit def toH2OFrameKeyFromFrame[T <: Frame](fr: T): Key[Frame] = fr._key + implicit def toH2OFrameKeyFromH2OFrame(fr: H2OFrame): Key[Frame] = fr.key /** Transform given Scala symbol to String */ implicit def symbolToString(sy: scala.Symbol): String = sy.name diff --git a/core/src/main/scala/org/apache/spark/h2o/H2ORelation.scala b/core/src/main/scala/org/apache/spark/h2o/H2ORelation.scala index 48bfae2aa..651fd8cd5 100644 --- a/core/src/main/scala/org/apache/spark/h2o/H2ORelation.scala +++ b/core/src/main/scala/org/apache/spark/h2o/H2ORelation.scala @@ -17,6 +17,7 @@ package org.apache.spark.h2o +import org.apache.spark.h2o.utils.H2OSchemaUtils import org.apache.spark.sql.DataFrame import org.apache.spark.sql.types.StructType import water.DKV @@ -27,8 +28,8 @@ object DataSourceUtils { H2OSchemaUtils.createSchema(frame, copyMetadata) } - def overwrite(key: String,originalFrame: H2OFrame, newDataFrame: DataFrame)(implicit h2oContext: H2OContext): Unit = { + def overwrite(key: String, originalFrame: Frame, newDataFrame: DataFrame)(implicit h2oContext: H2OContext): Unit = { originalFrame.remove() - h2oContext.asH2OFrame(newDataFrame,key) + h2oContext.asH2OFrame(newDataFrame, key) } } diff --git a/core/src/main/scala/org/apache/spark/h2o/JavaH2OContext.java b/core/src/main/scala/org/apache/spark/h2o/JavaH2OContext.java new file mode 100644 index 000000000..a60f1da6b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/h2o/JavaH2OContext.java @@ -0,0 +1,314 @@ +/* +* 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.h2o; + +import org.apache.spark.SparkContext; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.h2o.converters.SupportedRDD$; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.SQLContext; +import scala.Option; +import water.Key; +import water.fvec.Frame; +import water.fvec.H2OFrame; + +/** + * A Java-friendly version of [[org.apache.spark.h2o.H2OContext]] + * + * + * Sparkling Water can run in two modes. External cluster mode and internal cluster mode. When using external cluster + * mode, it tries to connect to existing H2O cluster using the provided spark + * configuration properties. In the case of internal cluster mode,it creates H2O cluster living in Spark - that means + * that each Spark executor will have one h2o instance running in it. This mode is not + * recommended for big clusters and clusters where Spark executors are not stable. + * + * Cluster mode can be set using the spark configuration + * property spark.ext.h2o.mode which can be set in script starting sparkling-water or + * can be set in H2O configuration class H2OConf + * + */ +public class JavaH2OContext { +/* +Note for developers: This class is not written in scala intentionally as we want to have static method getOrCreate on +the class itself and not on class generated from the object ( like H2OContext$). This way the functionality and API +remains the same as in H2OContext, but we need to write a few pass-through functions. + +If we write this class in scala the Java users would have to call getOrCreate method on generated class ending with $ +which is not nice. + */ + private H2OContext hc; + + public H2OContext h2oContext(){ + return hc; + } + + public SparkContext sparkContext(){ + return hc.sparkContext(); + } + /** + * Create new JavaH2OContext based on existing H2O Context + * + * @param hc H2O Context + */ + private JavaH2OContext(H2OContext hc){ + this.hc = hc; + } + + /** + * Get Java H2O Context based on existing H2O Context + * @param hc H2O Context + * @return Java H2O Context + */ + public static JavaH2OContext getFromExisting(H2OContext hc){ + return new JavaH2OContext(hc); + } + + + /** Pass-through to H2OContext.asH2OFrame. For API support only.*/ + public H2OFrame asH2OFrame(DataFrame df){ + return hc.asH2OFrame(df); + } + + /** Pass-through to H2OContext.asH2OFrame. For API support only.*/ + public H2OFrame asH2OFrame(DataFrame df, String frameName){ + return hc.asH2OFrame(df, frameName); + } + + + /** Create a new H2OFrame based on existing Frame referenced by its key.*/ + public H2OFrame asH2OFrame(String s){ + return hc.asH2OFrame(s); + } + + /** Create a new H2OFrame based on existing Frame */ + public H2OFrame asH2OFrame(Frame fr){ + return hc.asH2OFrame(fr); + } + + /** Convert given H2O frame into a Product RDD type */ + public JavaRDD asRDD(H2OFrame fr){ + //TODO: Implement this conversion + //return hc.asRDD(fr, (RDD)JavaSparkContext.fakeClassTag()) + return null; + } + + /** Convert given H2O frame into DataFrame type */ + public DataFrame asDataFrame(Frame fr, SQLContext sqlContext){ + return asDataFrame(fr, true, sqlContext); + } + /** Convert given H2O frame into DataFrame type */ + public DataFrame asDataFrame(Frame fr, boolean copyMetadata, SQLContext sqlContext){ + return hc.asDataFrame(fr, copyMetadata, sqlContext); + } + + + /** Convert given H2O frame into DataFrame type */ + public DataFrame asDataFrame(String key, SQLContext sqlContext){ + return asDataFrame(key, true, sqlContext); + } + /** Convert given H2O frame into DataFrame type */ + public DataFrame asDataFrame(String key, boolean copyMetadata, SQLContext sqlContext){ + return hc.asDataFrame(key, copyMetadata, sqlContext); + } + + /** Pass-through to H2OContext.toH2OFrameKey. For API support only.*/ + public Key toH2OFrameKey(DataFrame df){ + return hc.toH2OFrameKey(df); + } + + /** Pass-through to H2OContext.toH2OFrameKey. For API support only.*/ + public Key toH2OFrameKey(DataFrame df, Option frameName){ + return hc.toH2OFrameKey(df, frameName); + } + + /** Pass-through to H2OContext.toH2OFrameKey. For API support only.*/ + public Key toH2OFrameKey(DataFrame df, String frameName){ + return hc.toH2OFrameKey(df, frameName); + } + + /** + * Get existing or create new JavaH2OContext based on provided H2O configuration. It searches the configuration + * properties passed to Sparkling Water and based on them starts H2O Context. If the values are not found, the default + * values are used in most of the cases. The default cluster mode is internal, ie. spark.ext.h2o.external.cluster.mode=false + * + * @param jsc Java Spark Context + * @return Java H2O Context + */ + public static JavaH2OContext getOrCreate(JavaSparkContext jsc){ + H2OConf conf = new H2OConf(jsc.sc()); + return new JavaH2OContext(H2OContext.getOrCreate(jsc.sc(), conf)); + } + + /** + * Get existing or create new JavaH2OContext based on provided H2O configuration + * + * @param jsc Java Spark Context + * @param conf H2O configuration + * @return Java H2O Context + */ + public static JavaH2OContext getOrCreate(JavaSparkContext jsc, H2OConf conf){ + return new JavaH2OContext(H2OContext.getOrCreate(jsc.sc(), conf)); + } + + + public String toString(){ + return hc.toString(); + } + + public String h2oLocalClient(){ + return hc.h2oLocalClient(); + } + + public String h2oLocalClientIp(){ + return hc.h2oLocalClientIp(); + } + + public int h2oLocalClientPort(){ + return hc.h2oLocalClientPort(); + } + + public void stop(boolean stopSparkContext){ + hc.stop(stopSparkContext); + } + + public void openFlow(){ + hc.openFlow(); + } + /** + * Return a copy of this JavaH2OContext's configuration. The configuration ''cannot'' be changed at runtime. + */ + public H2OConf getConf(){ + return hc.getConf(); + } + + /** Conversion from RDD[String] to H2O's DataFrame */ + public H2OFrame asH2OFrameFromRDDString(JavaRDD rdd, String frameName){ + return hc.asH2OFrame(SupportedRDD$.MODULE$.toH2OFrameFromRDDString(rdd.rdd()), Option.apply(frameName)); + } + + /** Conversion from RDD[Boolean] to H2O's DataFrame */ + public H2OFrame asH2OFrameFromRDDBool(JavaRDD rdd, String frameName){ + + return hc.asH2OFrame(SupportedRDD$.MODULE$.toH2OFrameFromRDDJavaBool(rdd.rdd()), Option.apply(frameName)); + } + + /** Conversion from RDD[Integer] to H2O's DataFrame */ + public H2OFrame asH2OFrameFromRDDInt(JavaRDD rdd, String frameName){ + + return hc.asH2OFrame(SupportedRDD$.MODULE$.toH2OFrameFromRDDJavaInt(rdd.rdd()), Option.apply(frameName)); + } + + /** Conversion from RDD[Byte] to H2O's DataFrame */ + public H2OFrame asH2OFrameFromRDDByte(JavaRDD rdd, String frameName){ + + return hc.asH2OFrame(SupportedRDD$.MODULE$.toH2OFrameFromRDDJavaByte(rdd.rdd()), Option.apply(frameName)); + } + + /** Conversion from RDD[Short] to H2O's DataFrame */ + public H2OFrame asH2OFrameFromRDDShort(JavaRDD rdd, String frameName){ + + return hc.asH2OFrame(SupportedRDD$.MODULE$.toH2OFrameFromRDDJavaShort(rdd.rdd()), Option.apply(frameName)); + } + + /** Conversion from RDD[Float] to H2O's DataFrame */ + public H2OFrame asH2OFrameFromRDDFloat(JavaRDD rdd, String frameName){ + + return hc.asH2OFrame(SupportedRDD$.MODULE$.toH2OFrameFromRDDJavaFloat(rdd.rdd()), Option.apply(frameName)); + } + + /** Conversion from RDD[Double] to H2O's DataFrame */ + public H2OFrame asH2OFrameFromRDDDouble(JavaRDD rdd, String frameName){ + return hc.asH2OFrame(SupportedRDD$.MODULE$.toH2OFrameFromRDDJavaDouble(rdd.rdd()), Option.apply(frameName)); + } + + /** Conversion from RDD[Long] to H2O's DataFrame */ + public H2OFrame asH2OFrameFromRDDLong(JavaRDD rdd, String frameName){ + return hc.asH2OFrame(SupportedRDD$.MODULE$.toH2OFrameFromRDDJavaLong(rdd.rdd()), Option.apply(frameName)); + } + + /** Conversion from RDD[LabeledPoint] to H2O's DataFrame */ + public H2OFrame asH2OFrameFromRDDLabeledPoint(JavaRDD rdd, String frameName){ + return hc.asH2OFrame(SupportedRDD$.MODULE$.toH2OFrameFromRDDLabeledPoint(rdd.rdd()), Option.apply(frameName)); + } + + /** Conversion from RDD[java.sql.TimeStamp] to H2O's DataFrame */ + public H2OFrame asH2OFrameFromRDDTimeStamp(JavaRDD rdd, String frameName){ + return hc.asH2OFrame(SupportedRDD$.MODULE$.toH2OFrameFromRDDTimeStamp(rdd.rdd()), Option.apply(frameName)); + } + /** Returns key of the H2O's DataFrame conversed from RDD[String]*/ + @SuppressWarnings("unchecked") + public Key asH2OFrameFromRDDStringKey(JavaRDD rdd, String frameName){ + return (Key) hc.toH2OFrameKey(SupportedRDD$.MODULE$.toH2OFrameFromRDDString(rdd.rdd()), Option.apply(frameName)); + } + + /** Returns key of the H2O's DataFrame conversed from RDD[Boolean]*/ + @SuppressWarnings("unchecked") + public Key asH2OFrameFromRDDBoolKey(JavaRDD rdd, String frameName){ + return (Key) hc.toH2OFrameKey(SupportedRDD$.MODULE$.toH2OFrameFromRDDJavaBool(rdd.rdd()), Option.apply(frameName)); + } + + /** Returns key of the H2O's DataFrame conversed from RDD[Integer]*/ + @SuppressWarnings("unchecked") + public Key asH2OFrameFromRDDIntKey(JavaRDD rdd, String frameName){ + return (Key) hc.toH2OFrameKey(SupportedRDD$.MODULE$.toH2OFrameFromRDDJavaInt(rdd.rdd()), Option.apply(frameName)); + } + + /** Returns key of the H2O's DataFrame conversed from RDD[Byte]*/ + @SuppressWarnings("unchecked") + public Key asH2OFrameFromRDDByteKey(JavaRDD rdd, String frameName){ + return (Key) hc.toH2OFrameKey(SupportedRDD$.MODULE$.toH2OFrameFromRDDJavaByte(rdd.rdd()), Option.apply(frameName)); + } + + /** Returns key of the H2O's DataFrame conversed from RDD[Short]*/ + @SuppressWarnings("unchecked") + public Key asH2OFrameFromRDDShortKey(JavaRDD rdd, String frameName){ + return (Key) hc.toH2OFrameKey(SupportedRDD$.MODULE$.toH2OFrameFromRDDJavaShort(rdd.rdd()), Option.apply(frameName)); + } + + /** Returns key of the H2O's DataFrame conversed from RDD[Float]*/ + @SuppressWarnings("unchecked") + public Key asH2OFrameFromRDDFloatKey(JavaRDD rdd, String frameName){ + return (Key) hc.toH2OFrameKey(SupportedRDD$.MODULE$.toH2OFrameFromRDDJavaFloat(rdd.rdd()), Option.apply(frameName)); + } + + /** Returns key of the H2O's DataFrame conversed from RDD[Double]*/ + @SuppressWarnings("unchecked") + public Key asH2OFrameFromRDDDoubleKey(JavaRDD rdd, String frameName) { + return (Key) hc.toH2OFrameKey(SupportedRDD$.MODULE$.toH2OFrameFromRDDJavaDouble(rdd.rdd()), Option.apply(frameName)); + } + + /** Returns key of the H2O's DataFrame conversed from RDD[Long]*/ + @SuppressWarnings("unchecked") + public Key asH2OFrameFromRDDLongKey(JavaRDD rdd, String frameName){ + return (Key) hc.toH2OFrameKey(SupportedRDD$.MODULE$.toH2OFrameFromRDDJavaLong(rdd.rdd()), Option.apply(frameName)); + } + + /** Returns key of the H2O's DataFrame conversed from RDD[LabeledPoint]*/ + @SuppressWarnings("unchecked") + public Key asH2OFrameFromRDDLabeledPointKey(JavaRDD rdd, String frameName){ + return (Key) hc.toH2OFrameKey(SupportedRDD$.MODULE$.toH2OFrameFromRDDLabeledPoint(rdd.rdd()), Option.apply(frameName)); + } + + /** Returns key of the H2O's DataFrame conversed from RDD[java.sql.Timestamp]*/ + @SuppressWarnings("unchecked") + public Key asH2OFrameFromRDDTimeStampKey(JavaRDD rdd, String frameName){ + return (Key) hc.toH2OFrameKey(SupportedRDD$.MODULE$.toH2OFrameFromRDDTimeStamp(rdd.rdd()), Option.apply(frameName)); + } +} diff --git a/core/src/main/scala/org/apache/spark/h2o/SupportedRDD.scala b/core/src/main/scala/org/apache/spark/h2o/SupportedRDD.scala deleted file mode 100644 index c07e28090..000000000 --- a/core/src/main/scala/org/apache/spark/h2o/SupportedRDD.scala +++ /dev/null @@ -1,79 +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.h2o - -import org.apache.spark.SparkContext -import org.apache.spark.mllib.regression.LabeledPoint - -import scala.language.implicitConversions -import scala.reflect.runtime.universe._ - - -/** - * Magnet pattern (Type Class pattern) for conversion from various primitive types to their appropriate H2OFrame using - * the method with the same name - */ -trait SupportedRDD { - def toH2OFrame(sc: SparkContext, frameKeyName: Option[String]): H2OFrame -} - -object SupportedRDD { - - implicit def toH2OFrameFromRDDString(rdd: RDD[String]): SupportedRDD = new SupportedRDD { - override def toH2OFrame(sc: SparkContext, frameKeyName: Option[String]): H2OFrame = H2OContext.toH2OFrameFromRDDString(sc, rdd, frameKeyName) - } - - implicit def toH2OFrameFromRDDInt(rdd: RDD[Int]): SupportedRDD = new SupportedRDD { - override def toH2OFrame(sc: SparkContext, frameKeyName: Option[String]): H2OFrame = H2OContext.toH2OFrameFromRDDInt(sc, rdd, frameKeyName) - } - - implicit def toH2OFrameFromRDDByte(rdd: RDD[Byte]): SupportedRDD = new SupportedRDD { - override def toH2OFrame(sc: SparkContext, frameKeyName: Option[String]): H2OFrame = H2OContext.toH2OFrameFromRDDByte(sc, rdd, frameKeyName) - } - - implicit def toH2OFrameFromRDDShort(rdd: RDD[Short]): SupportedRDD = new SupportedRDD { - override def toH2OFrame(sc: SparkContext, frameKeyName: Option[String]): H2OFrame = H2OContext.toH2OFrameFromRDDShort(sc, rdd, frameKeyName) - } - - implicit def toH2OFrameFromRDDFloat(rdd: RDD[Float]): SupportedRDD = new SupportedRDD { - override def toH2OFrame(sc: SparkContext, frameKeyName: Option[String]): H2OFrame = H2OContext.toH2OFrameFromRDDFloat(sc, rdd, frameKeyName) - } - - implicit def toH2OFrameFromDouble(rdd: RDD[Double]): SupportedRDD = new SupportedRDD { - override def toH2OFrame(sc: SparkContext, frameKeyName: Option[String]): H2OFrame = H2OContext.toH2OFrameFromRDDDouble(sc, rdd, frameKeyName) - } - - implicit def toH2OFrameFromRDDBool(rdd: RDD[Boolean]): SupportedRDD = new SupportedRDD { - override def toH2OFrame(sc: SparkContext, frameKeyName: Option[String]): H2OFrame = H2OContext.toH2OFrameFromRDDBool(sc, rdd, frameKeyName) - } - - implicit def toH2OFrameFromRDDLong(rdd: RDD[Long]): SupportedRDD = new SupportedRDD { - override def toH2OFrame(sc: SparkContext, frameKeyName: Option[String]): H2OFrame = H2OContext.toH2OFrameFromRDDLong(sc, rdd, frameKeyName) - } - - implicit def toH2OFrameFromRDDLabeledPoint(rdd: RDD[LabeledPoint]): SupportedRDD = new SupportedRDD { - override def toH2OFrame(sc: SparkContext, frameKeyName: Option[String]): H2OFrame = H2OContext.toH2OFrameFromRDDLabeledPoint(sc, rdd, frameKeyName) - } - - implicit def toH2OFrameFromRDDTimeStamo(rdd: RDD[java.sql.Timestamp]): SupportedRDD = new SupportedRDD { - override def toH2OFrame(sc: SparkContext, frameKeyName: Option[String]): H2OFrame = H2OContext.toH2OFrameFromRDDTimeStamp(sc, rdd, frameKeyName) - } - - implicit def toH2OFrameFromRDDProduct[A <: Product : TypeTag](rdd : RDD[A]): SupportedRDD = new SupportedRDD { - override def toH2OFrame(sc: SparkContext, frameKeyName: Option[String]): H2OFrame = H2OContext.toH2OFrame(sc, rdd, frameKeyName) - } -} diff --git a/core/src/main/scala/org/apache/spark/h2o/backends/SharedBackendUtils.scala b/core/src/main/scala/org/apache/spark/h2o/backends/SharedBackendUtils.scala new file mode 100644 index 000000000..b09fb93a7 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/h2o/backends/SharedBackendUtils.scala @@ -0,0 +1,164 @@ +/* +* 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.h2o.backends + +import java.io.File + +import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.h2o.H2OConf + +/** + * Shared functions which can be used by both backends + */ +private[backends] trait SharedBackendUtils extends Logging with Serializable{ + + /** + * Return hostname of this node based on SparkEnv + * + * @param env SparkEnv instance + * @return hostname of the node + */ + def getHostname(env: SparkEnv) = env.blockManager.blockManagerId.host + + /** Check Spark and H2O environment, update it if necessary and and warn about possible problems. + * + * This method checks the environments for generic configuration which does not depend on particular backend used + * In order to check the configuration for specific backend, method checkAndUpdateConf on particular backend has to be + * called. + * + * This method has to be called at the start of each method which override this one + * + * @param conf H2O Configuration to check + * @return checked and updated configuration + * */ + def checkAndUpdateConf(conf: H2OConf): H2OConf = { + // Note: updating Spark Conf is useless at this time in more of the cases since SparkContext is already running + + // Increase locality timeout since h2o-specific tasks can be long computing + if (conf.getInt("spark.locality.wait", 3000) <= 3000) { + logWarning(s"Increasing 'spark.locality.wait' to value 30000") + conf.set("spark.locality.wait", "30000") + } + + if(conf.clientIp.isEmpty){ + conf.setClientIp(getHostname(SparkEnv.get)) + } + + if(conf.backendClusterMode != "internal" && conf.backendClusterMode != "external"){ + logWarning( + s"""'spark.ext.h2o.backend.cluster.mode' property is set to ${conf.backendClusterMode}. + Valid options are "internal" or "external". Running in internal cluster mode now! + """) + + } + + conf + } + + /** + * Get H2O arguments which are passed to every node - regular node, client node + * + * @param conf H2O Configuration + * @return sequence of arguments + */ + def getH2OCommonArgs(conf: H2OConf): Seq[String] = + // Options in form key=value + Seq( + ("-name", conf.cloudName.get), + ("-nthreads", if (conf.nthreads > 0) conf.nthreads else null)) + .filter(x => x._2 != null) + .flatMap(x => Seq(x._1, x._2.toString)) ++ // Append single boolean options + Seq(("-ga_opt_out", conf.disableGA)) + .filter(_._2).map(x => x._1) + + + /** + * Get common arguments for H2O client. + * + * @return array of H2O client arguments. + */ + def getH2OClientArgs(conf: H2OConf): Array[String] = ( + getH2OCommonArgs(conf) + ++ (if (!conf.clientVerboseOutput) Seq("-quiet") else Nil) + ++ (if (conf.hashLogin) Seq("-hash_login") else Nil) + ++ (if (conf.ldapLogin) Seq("-ldap_login") else Nil) + ++ (if (conf.kerberosLogin) Seq("-kerberos_login") else Nil) + ++ Seq("-log_level", if (conf.clientVerboseOutput) incLogLevel(conf.h2oClientLogLevel, "INFO") else conf.h2oClientLogLevel) + ++ Seq("-log_dir", conf.h2oClientLogDir) + ++ Seq("-baseport", conf.clientBasePort.toString) + ++ Seq("-client") + ++ Seq( + ("-ice_root", conf.clientIcedDir.orNull), + ("-port", if (conf.clientWebPort > 0) conf.clientWebPort else null), + ("-jks", conf.jks.orNull), + ("-jks_pass", conf.jksPass.orNull), + ("-login_conf", conf.loginConf.orNull), + ("-user_name", conf.userName.orNull), + conf.clientNetworkMask.map(mask => ("-network", mask)).getOrElse(("-ip", conf.clientIp.get)) + ).filter(_._2 != null).flatMap(x => Seq(x._1, x._2.toString)) + ).toArray + + val TEMP_DIR_ATTEMPTS = 1000 + + private def createTempDir(): File = { + def baseDir = new File(System.getProperty("java.io.tmpdir")) + def baseName = System.currentTimeMillis() + "-" + + var cnt = 0 + while (cnt < TEMP_DIR_ATTEMPTS) { + // infinite loop + val tempDir = new File(baseDir, baseName + cnt) + if (tempDir.mkdir()) return tempDir + cnt += 1 + } + throw new IllegalStateException(s"Failed to create temporary directory ${baseDir} / ${baseName}") + } + + def saveAsFile(content: String): File = { + val tmpDir = createTempDir() + tmpDir.deleteOnExit() + val flatFile = new File(tmpDir, "flatfile.txt") + val p = new java.io.PrintWriter(flatFile) + try { + p.print(content) + } finally { + p.close() + } + flatFile + } + + /** + * Increment log level to at least desired minimal log level. + * @param logLevel actual log level + * @param minLogLevel desired minimal log level + * @return if logLevel is less verbose than minLogLeve then minLogLevel, else logLevel + */ + private def incLogLevel(logLevel: String, minLogLevel: String): String = { + val logLevels = Seq( ("OFF", 0), ("FATAL", 1), ("ERROR", 2), + ("WARN", 3), ("INFO", 4), ("DEBUG", 5), ("TRACE", 6), ("ALL", 7)) + val ll = logLevels.find(t => t._1 == logLevel) + val mll = logLevels.find(t => t._1 == minLogLevel) + if (mll.isEmpty) { + logLevel + } else { + ll.map(v => if (v._2 < mll.get._2) minLogLevel else logLevel).getOrElse(minLogLevel) + } + } +} + +object SharedBackendUtils extends SharedBackendUtils diff --git a/core/src/main/scala/org/apache/spark/h2o/backends/SharedH2OConf.scala b/core/src/main/scala/org/apache/spark/h2o/backends/SharedH2OConf.scala new file mode 100644 index 000000000..41588aaea --- /dev/null +++ b/core/src/main/scala/org/apache/spark/h2o/backends/SharedH2OConf.scala @@ -0,0 +1,178 @@ +/* +* 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.h2o.backends + +import org.apache.spark.h2o.H2OConf + +/** + * Shared configuration independent on used backend + */ +trait SharedH2OConf { + self: H2OConf => + + import SharedH2OConf._ + + def backendClusterMode = sparkConf.get(PROP_BACKEND_CLUSTER_MODE._1, PROP_BACKEND_CLUSTER_MODE._2) + + def clientIp = sparkConf.getOption(PROP_CLIENT_IP._1) + def clientVerboseOutput = sparkConf.getBoolean(PROP_CLIENT_VERBOSE._1, PROP_CLIENT_VERBOSE._2) + def clientBasePort = sparkConf.getInt(PROP_CLIENT_PORT_BASE._1, PROP_CLIENT_PORT_BASE._2) + def cloudName = sparkConf.getOption(PROP_CLOUD_NAME._1) + def h2oClientLogLevel = sparkConf.get(PROP_CLIENT_LOG_LEVEL._1, PROP_CLIENT_LOG_LEVEL._2) + def h2oClientLogDir = sparkConf.get(PROP_CLIENT_LOG_DIR._1, PROP_CLIENT_LOG_DIR._2) + def clientNetworkMask = sparkConf.getOption(PROP_CLIENT_NETWORK_MASK._1) + def nthreads = sparkConf.getInt(PROP_NTHREADS._1, PROP_NTHREADS._2) + def disableGA = sparkConf.getBoolean(PROP_DISABLE_GA._1, PROP_DISABLE_GA._2) + def clientWebPort = sparkConf.getInt(PROP_CLIENT_WEB_PORT._1, PROP_CLIENT_WEB_PORT._2) + def clientIcedDir = sparkConf.getOption(PROP_CLIENT_ICED_DIR._1) + + def jks = sparkConf.getOption(PROP_JKS._1) + def jksPass = sparkConf.getOption(PROP_JKS_PASS._1) + def hashLogin = sparkConf.getBoolean(PROP_HASH_LOGIN._1, PROP_HASH_LOGIN._2) + def ldapLogin = sparkConf.getBoolean(PROP_LDAP_LOGIN._1, PROP_LDAP_LOGIN._2) + def kerberosLogin = sparkConf.getBoolean(PROP_KERBEROS_LOGIN._1, PROP_KERBEROS_LOGIN._2) + def loginConf = sparkConf.getOption(PROP_LOGIN_CONF._1) + def userName = sparkConf.getOption(PROP_USER_NAME._1) + + def isFailOnUnsupportedSparkParamEnabled = sparkConf.getBoolean(PROP_FAIL_ON_UNSUPPORTED_SPARK_PARAM._1, PROP_FAIL_ON_UNSUPPORTED_SPARK_PARAM._2) + def scalaIntDefaultNum = sparkConf.getInt(PROP_SCALA_INT_DEFAULT_NUM._1, PROP_SCALA_INT_DEFAULT_NUM._2) + def isH2OReplEnabled = sparkConf.getBoolean(PROP_REPL_ENABLED._1, PROP_REPL_ENABLED._2) + def isClusterTopologyListenerEnabled = sparkConf.getBoolean(PROP_CLUSTER_TOPOLOGY_LISTENER_ENABLED._1, PROP_CLUSTER_TOPOLOGY_LISTENER_ENABLED._2) + def isSparkVersionCheckEnabled = sparkConf.getBoolean(PROP_SPARK_VERSION_CHECK_ENABLED._1, PROP_SPARK_VERSION_CHECK_ENABLED._2) + + def runsInExternalClusterMode: Boolean = backendClusterMode.toLowerCase() == "external" + def runsInInternalClusterMode: Boolean = !runsInInternalClusterMode + + def setCloudName(cloudName: String): H2OConf = { + sparkConf.set(PROP_CLOUD_NAME._1, cloudName) + self + } + + private[this] def setBackendClusterMode(backendClusterMode: String): H2OConf = { + sparkConf.set(PROP_BACKEND_CLUSTER_MODE._1, backendClusterMode) + self + } + + def setInternalClusterMode(): H2OConf = { + setBackendClusterMode("internal") + self + } + + def setReplDisabled(): H2OConf = { + sparkConf.set(PROP_REPL_ENABLED._1, false.toString) + self + } + def setReplEnabled(): H2OConf = { + sparkConf.set(PROP_REPL_ENABLED._1, true.toString) + self + } + + def setExternalClusterMode(): H2OConf = { + setBackendClusterMode("external") + self + } + + def setClientIp(ip: String): H2OConf = { + sparkConf.set(PROP_CLIENT_IP._1, ip) + self + } +} + +object SharedH2OConf { + /** + * This option can be set either to "internal" or "external" + * When set to "external" H2O Context is created by connecting to existing H2O cluster, otherwise it creates + * H2O cluster living in Spark - that means that each Spark executor will have one h2o instance running in it. + * The internal is not recommended for big clusters and clusters where Spark executors are not stable. + */ + val PROP_BACKEND_CLUSTER_MODE = ("spark.ext.h2o.backend.cluster.mode", "internal") + + /** IP of H2O client node */ + val PROP_CLIENT_IP = ("spark.ext.h2o.client.ip", null.asInstanceOf[String]) + + /** Print detailed messages to client stdout */ + val PROP_CLIENT_VERBOSE = ("spark.ext.h2o.client.verbose", false) + + /** Port on which H2O client publishes its API. If already occupied, the next odd port is tried and so on */ + val PROP_CLIENT_PORT_BASE = ( "spark.ext.h2o.client.port.base", 54321 ) + + /** Configuration property - name of H2O cloud */ + val PROP_CLOUD_NAME = ("spark.ext.h2o.cloud.name", null.asInstanceOf[String]) + + /** H2O log leve for client running in Spark driver */ + val PROP_CLIENT_LOG_LEVEL = ("spark.ext.h2o.client.log.level", "WARN") + + /** Location of log directory for the driver instance. */ + val PROP_CLIENT_LOG_DIR = ("spark.ext.h2o.client.log.dir", defaultLogDir) + + /** Subnet selector for H2O client - if the mask is specified then Spark network setup is not discussed. */ + val PROP_CLIENT_NETWORK_MASK = ("spark.ext.h2o.client.network.mask", null.asInstanceOf[String]) + + /** Limit for number of threads used by H2O, default -1 means unlimited */ + val PROP_NTHREADS = ("spark.ext.h2o.nthreads", -1) + + /** Disable GA tracking */ + val PROP_DISABLE_GA = ("spark.ext.h2o.disable.ga", true) + + /** Exact client port to access web UI. + * The value `-1` means automatic search for free port starting at `spark.ext.h2o.port.base`. */ + val PROP_CLIENT_WEB_PORT = ("spark.ext.h2o.client.web.port", -1) + + /** Location of iced directory for the driver instance. */ + val PROP_CLIENT_ICED_DIR = ("spark.ext.h2o.client.iced.dir", null.asInstanceOf[String]) + + /** Path to Java KeyStore file. */ + val PROP_JKS = ("spark.ext.h2o.jks", null.asInstanceOf[String]) + + /** Password for Java KeyStore file. */ + val PROP_JKS_PASS = ("spark.ext.h2o.jks.pass", null.asInstanceOf[String]) + + /** Enable hash login. */ + val PROP_HASH_LOGIN = ("spark.ext.h2o.hash.login", false) + + /** Enable LDAP login. */ + val PROP_LDAP_LOGIN = ("spark.ext.h2o.ldap.login", false) + + /** Enable Kerberos login. */ + val PROP_KERBEROS_LOGIN = ("spark.ext.h2o.kerberos.login", false) + + /** Login configuration file. */ + val PROP_LOGIN_CONF = ("spark.ext.h2o.login.conf", null.asInstanceOf[String]) + + /** Override user name for cluster. */ + val PROP_USER_NAME = ("spark.ext.h2o.user.name", null.asInstanceOf[String]) + + /** Number of executors started at the start of h2o services, by default 1 */ + val PROP_SCALA_INT_DEFAULT_NUM = ("spark.ext.scala.int.default.num", 1) + + /** Enable/Disable Sparkling-Water REPL **/ + val PROP_REPL_ENABLED = ("spark.ext.h2o.repl.enabled", true) + + /** Enable/Disable listener which kills H2O when there is a change in underlying cluster's topology**/ + val PROP_CLUSTER_TOPOLOGY_LISTENER_ENABLED = ("spark.ext.h2o.topology.change.listener.enabled", true) + + /** Enable/Disable check for Spark version. */ + val PROP_SPARK_VERSION_CHECK_ENABLED = ("spark.ext.h2o.spark.version.check.enabled", true) + + /** Enable/Disable exit on unsupported Spark parameters. */ + val PROP_FAIL_ON_UNSUPPORTED_SPARK_PARAM = ("spark.ext.h2o.fail.on.unsupported.spark.param", true) + + private[spark] def defaultLogDir: String = { + System.getProperty("user.dir") + java.io.File.separator + "h2ologs" + } +} diff --git a/core/src/main/scala/org/apache/spark/h2o/backends/SparklingBackend.scala b/core/src/main/scala/org/apache/spark/h2o/backends/SparklingBackend.scala new file mode 100644 index 000000000..c22a5a8d2 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/h2o/backends/SparklingBackend.scala @@ -0,0 +1,35 @@ +/* +* 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.h2o.backends + +import org.apache.spark.h2o.H2OConf +import org.apache.spark.h2o.utils.NodeDesc + +trait SparklingBackend { + + def init(): Array[NodeDesc] + + /** + * Check Spark and H2O environment on particular backend, update it if necessary and and warn about possible problems + * + * @param conf H2O Configuration + */ + def checkAndUpdateConf(conf: H2OConf): H2OConf + + def stop(stopSparkContext: Boolean) +} diff --git a/core/src/main/scala/org/apache/spark/h2o/backends/internal/InternalBackendConf.scala b/core/src/main/scala/org/apache/spark/h2o/backends/internal/InternalBackendConf.scala new file mode 100644 index 000000000..6f209f3b2 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/h2o/backends/internal/InternalBackendConf.scala @@ -0,0 +1,97 @@ +/* +* 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.h2o.backends.internal + +import org.apache.spark.h2o.H2OConf +import org.apache.spark.h2o.backends.SharedH2OConf + +/** + * Internal backend configuration + */ +trait InternalBackendConf extends SharedH2OConf { + self: H2OConf => + + import InternalBackendConf._ + def numH2OWorkers = sparkConf.getOption(PROP_CLUSTER_SIZE._1).map(_.toInt) + def useFlatFile = sparkConf.getBoolean(PROP_USE_FLATFILE._1, PROP_USE_FLATFILE._2) + def nodeBasePort = sparkConf.getInt(PROP_NODE_PORT_BASE._1, PROP_NODE_PORT_BASE._2) + def cloudTimeout = sparkConf.getInt(PROP_CLOUD_TIMEOUT._1, PROP_CLOUD_TIMEOUT._2) + def drddMulFactor = sparkConf.getInt(PROP_DUMMY_RDD_MUL_FACTOR._1, PROP_DUMMY_RDD_MUL_FACTOR._2) + def numRddRetries = sparkConf.getInt(PROP_SPREADRDD_RETRIES._1, PROP_SPREADRDD_RETRIES._2) + def defaultCloudSize = sparkConf.getInt(PROP_DEFAULT_CLUSTER_SIZE._1, PROP_DEFAULT_CLUSTER_SIZE._2) + def h2oNodeLogLevel = sparkConf.get(PROP_NODE_LOG_LEVEL._1, PROP_NODE_LOG_LEVEL._2) + def h2oNodeLogDir = sparkConf.get(PROP_NODE_LOG_DIR._1, PROP_NODE_LOG_DIR._2) + def nodeNetworkMask = sparkConf.getOption(PROP_NODE_NETWORK_MASK._1) + def nodeIcedDir = sparkConf.getOption(PROP_NODE_ICED_DIR._1) + def subseqTries = sparkConf.getInt(PROP_SUBSEQ_TRIES._1, PROP_SUBSEQ_TRIES._2) + + def internalConfString: String = + s"""Sparkling Water configuration: + | backend cluster mode : ${backendClusterMode} + | workers : ${numH2OWorkers} + | cloudName : ${cloudName.get} + | flatfile : ${useFlatFile} + | clientBasePort : ${clientBasePort} + | nodeBasePort : ${nodeBasePort} + | cloudTimeout : ${cloudTimeout} + | h2oNodeLog : ${h2oNodeLogLevel} + | h2oClientLog : ${h2oClientLogLevel} + | nthreads : ${nthreads} + | drddMulFactor : $drddMulFactor""".stripMargin +} + +object InternalBackendConf { + /** Configuration property - expected number of workers of H2O cloud. + * Value None means automatic detection of cluster size. + */ + val PROP_CLUSTER_SIZE = ("spark.ext.h2o.cluster.size", None) + + /** Configuration property - use flatfile for H2O cloud formation. */ + val PROP_USE_FLATFILE = ("spark.ext.h2o.flatfile", true) + + /** Configuration property - base port used for individual H2O nodes configuration. */ + val PROP_NODE_PORT_BASE = ( "spark.ext.h2o.node.port.base", 54321) + + /** Configuration property - timeout for cloud up. */ + val PROP_CLOUD_TIMEOUT = ("spark.ext.h2o.cloud.timeout", 60*1000) + + /** Configuration property - multiplication factor for dummy RDD generation. + * Size of dummy RDD is PROP_CLUSTER_SIZE*PROP_DUMMY_RDD_MUL_FACTOR */ + val PROP_DUMMY_RDD_MUL_FACTOR = ("spark.ext.h2o.dummy.rdd.mul.factor", 10) + + /** Configuration property - number of retries to create an RDD spread over all executors */ + val PROP_SPREADRDD_RETRIES = ("spark.ext.h2o.spreadrdd.retries", 10) + + /** Starting size of cluster in case that size is not explicitly passed */ + val PROP_DEFAULT_CLUSTER_SIZE = ("spark.ext.h2o.default.cluster.size", 20) + + /** H2O internal log level for launched remote nodes. */ + val PROP_NODE_LOG_LEVEL = ("spark.ext.h2o.node.log.level", "INFO") + + /** Location of log directory for remote nodes. */ + val PROP_NODE_LOG_DIR = ("spark.ext.h2o.node.log.dir", null.asInstanceOf[String]) + + /** Subnet selector for H2O nodes running inside executors - if the mask is specified then Spark network setup is not discussed. */ + val PROP_NODE_NETWORK_MASK = ("spark.ext.h2o.node.network.mask", null.asInstanceOf[String]) + + /** Location of iced directory for Spark nodes */ + val PROP_NODE_ICED_DIR = ("spark.ext.h2o.node.iced.dir", null.asInstanceOf[String]) + + /** Subsequent successful tries to figure out size of Spark cluster which are producing same number of nodes. */ + val PROP_SUBSEQ_TRIES = ("spark.ext.h2o.subseq.tries", 5) +} diff --git a/core/src/main/scala/org/apache/spark/h2o/H2OContextUtils.scala b/core/src/main/scala/org/apache/spark/h2o/backends/internal/InternalBackendUtils.scala similarity index 65% rename from core/src/main/scala/org/apache/spark/h2o/H2OContextUtils.scala rename to core/src/main/scala/org/apache/spark/h2o/backends/internal/InternalBackendUtils.scala index 46e5774e1..cd8a12d5c 100644 --- a/core/src/main/scala/org/apache/spark/h2o/H2OContextUtils.scala +++ b/core/src/main/scala/org/apache/spark/h2o/backends/internal/InternalBackendUtils.scala @@ -15,79 +15,78 @@ * limitations under the License. */ -package org.apache.spark.h2o +package org.apache.spark.h2o.backends.internal -import java.io.File import java.net.InetAddress -import org.apache.spark.h2o.H2OContextUtils._ +import org.apache.spark.h2o.backends.{SharedBackendUtils, SharedH2OConf} +import org.apache.spark.h2o.utils.{NodeDesc, ReflectionUtils} +import org.apache.spark.h2o.{H2OConf, RDD} import org.apache.spark.scheduler.cluster.YarnSchedulerBackend import org.apache.spark.scheduler.local.LocalBackend -import org.apache.spark.scheduler.{SparkListenerBlockManagerAdded, SparkListenerBlockManagerRemoved} -import org.apache.spark.{Accumulable, SparkConf, SparkContext, SparkEnv} +import org.apache.spark.{Accumulable, SparkContext, SparkEnv} import water.H2OStarter import water.init.AbstractEmbeddedH2OConfig import scala.collection.mutable -/** - * Support methods for H2OContext. - */ -private[spark] object H2OContextUtils { - - /** Helper class containing ExecutorId, hostname and port */ - case class NodeDesc(executorId: String, hostname: String, port: Int){ - override def productPrefix = "" - } - /** - * Return hostname of this node based on SparkEnv - * @param env SparkEnv instance - * @return hostname of the node - */ - def getHostname(env: SparkEnv) = env.blockManager.blockManagerId.host - - def saveAsFile(content: String): File = { - val tmpDir = createTempDir() - tmpDir.deleteOnExit() - val flatFile = new File(tmpDir, "flatfile.txt") - val p = new java.io.PrintWriter(flatFile) - try { - p.print(content) - } finally { - p.close() - } - flatFile - } - - def toFlatFileString(executors: Array[NodeDesc]):String = { - executors.map(en => s"${en.hostname}:${en.port}").mkString("\n") +/** + * Various helper methods used in the internal backend + */ +private[internal] trait InternalBackendUtils extends SharedBackendUtils{ + + def checkUnsupportedSparkOptions(unsupportedSparkOptions: Seq[(String, String)], conf: H2OConf): Unit ={ + unsupportedSparkOptions.foreach(opt => if (conf.contains(opt._1) && (opt._2 == None || conf.get(opt._1) == opt._2)) { + logWarning(s"Unsupported options ${opt._1} detected!") + if (conf.isFailOnUnsupportedSparkParamEnabled) { + logWarning( + s""" + |The application is going down, since the parameter ${SharedH2OConf.PROP_FAIL_ON_UNSUPPORTED_SPARK_PARAM} is true! + |If you would like to skip the fail call, please, specify the value of the parameter to false. + """.stripMargin) + throw new IllegalArgumentException(s"Unsupported argument: ${opt}") + } + }) } - def toH2OArgs(h2oArgs: Array[String], h2oConf: H2OConf, executors: Array[NodeDesc]): Array[String] = { + def toH2OArgs(h2oArgs: Array[String], conf: H2OConf, executors: Array[NodeDesc]): Array[String] = { toH2OArgs( h2oArgs, - if (h2oConf.useFlatFile) Some(toFlatFileString(executors)) + if (conf.useFlatFile) Some(toFlatFileString(executors)) else None) } - def toH2OArgs(h2oArgs: Array[String], flatFileString: Option[String]): Array[String] = { + private[this] def toH2OArgs(h2oArgs: Array[String], flatFileString: Option[String]): Array[String] = { val launcherArgs = flatFileString .map(f => saveAsFile(f)) .map(f => h2oArgs ++ Array("-flatfile", f.getAbsolutePath)) .getOrElse(h2oArgs) launcherArgs } + /** + * Produce arguments for H2O node based on provided configuration + * + * @return array of H2O launcher command line arguments + */ + def getH2ONodeArgs(conf: H2OConf): Array[String] = (getH2OCommonArgs(conf) ++ + Seq("-log_level", conf.h2oNodeLogLevel, "-baseport", conf.nodeBasePort.toString)).toArray + + + def toFlatFileString(executors: Array[NodeDesc]):String = { + executors.map(en => s"${en.hostname}:${en.port}").mkString("\n") + } /** - * Start H2O nodes on given executors. - * - * @param sc Spark context - * @param spreadRDD helper RDD spread over all executors - * @param numOfExecutors number of executors in Spark cluster - * @param h2oArgs arguments passed to H2O instances - * @return flatfile string if flatfile mode is enabled, else None - */ + * Start H2O nodes on given executors. + * + * @param sc Spark context + * @param spreadRDD helper RDD spread over all executors + * @param numOfExecutors number of executors in Spark cluster + * @param h2oArgs arguments passed to H2O instances + * @param networkMask node network mask + * @return flatfile string if flatfile mode is enabled, else None + */ def startH2O( sc: SparkContext, spreadRDD: RDD[NodeDesc], numOfExecutors: Int, @@ -102,7 +101,7 @@ private[spark] object H2OContextUtils { // Try to launch H2O val executorStatus = spreadRDD.map { nodeDesc => // RDD partition index assert(nodeDesc.hostname == getHostname(SparkEnv.get), // Make sure we are running on right node - s"SpreadRDD failure - IPs are not equal: ${nodeDesc} != (${SparkEnv.get.executorId}, ${getHostname(SparkEnv.get)})") + s"SpreadRDD failure - IPs are not equal: ${nodeDesc} != (${SparkEnv.get.executorId}, ${getHostname(SparkEnv.get)})") // Launch the node val sparkEnv = SparkEnv.get // Define log dir @@ -111,12 +110,11 @@ private[spark] object H2OContextUtils { if (s != null) { return s + java.io.File.separator } - - if (sparkEnv.conf.contains(H2OConf.PROP_NODE_LOG_DIR._1)) { - sparkEnv.conf.get(H2OConf.PROP_NODE_LOG_DIR._1) + if (sparkEnv.conf.contains(InternalBackendConf.PROP_NODE_LOG_DIR._1)) { + sparkEnv.conf.get(InternalBackendConf.PROP_NODE_LOG_DIR._1) } else { // Needs to be executed at remote node! - H2OConf.defaultLogDir + SharedH2OConf.defaultLogDir } } val executorId = sparkEnv.executorId @@ -142,7 +140,7 @@ private[spark] object H2OContextUtils { H2OStarter.start(launcherArgs, !isLocal) // Signal via singleton object that h2o was started on this node H2OStartedSignal.synchronized { - H2OStartedSignal.setStarted + H2OStartedSignal.setStarted() H2OStartedSignal.notifyAll() } } @@ -159,7 +157,7 @@ private[spark] object H2OContextUtils { e.printStackTrace() println( s""""Cannot start H2O node because: ${e.getMessage} - | h2o parameters: ${h2oArgs.mkString(",")} + | h2o parameters: ${h2oArgs.mkString(",")} """.stripMargin) (executorId, false) } @@ -167,7 +165,7 @@ private[spark] object H2OContextUtils { }.collect() // The accumulable should contain all IP:PORTs from all exeuctors if (bc.value.size != numOfExecutors || - executorStatus.groupBy(_._1).flatMap( x => x._2.find(_._2)).size != numOfExecutors) { + executorStatus.groupBy(_._1).flatMap( x => x._2.find(_._2)).size != numOfExecutors) { throw new RuntimeException(s"Cannot launch H2O on executors: numOfExecutors=${numOfExecutors}, " + s"executorStatus=${executorStatus.mkString(",")}") } @@ -179,7 +177,6 @@ private[spark] object H2OContextUtils { spreadRDD.foreach { nodeDesc => val env = SparkEnv.get assert(nodeDesc.hostname == getHostname(env), s"nodeDesc=${nodeDesc} == ${getHostname(env)}") // Make sure we are running on right node - val executorId = env.executorId val econf = water.H2O.getEmbeddedH2OConfig().asInstanceOf[SparklingWaterConfig] // Setup flatfile for waiting guys @@ -196,21 +193,6 @@ private[spark] object H2OContextUtils { flatFile } - val TEMP_DIR_ATTEMPTS = 1000 - - private def createTempDir(): File = { - def baseDir = new File(System.getProperty("java.io.tmpdir")) - def baseName = System.currentTimeMillis() + "-" - - var cnt = 0 - while (cnt < TEMP_DIR_ATTEMPTS) {// infinite loop - val tempDir = new File(baseDir, baseName + cnt) - if (tempDir.mkdir()) return tempDir - cnt += 1 - } - throw new IllegalStateException(s"Failed to create temporary directory $baseDir / $baseName") - } - private[spark] def guessTotalExecutorSize(sc: SparkContext): Option[Int] = { sc.conf.getOption("spark.executor.instances") .map(_.toInt) @@ -230,10 +212,10 @@ private[spark] object H2OContextUtils { private def getCommandArg(argName: String): Option[String] = { val cmdLine = System.getProperty("sun.java.command", "").split(" ").map(_.trim) - val valueIdx = (for (i <- 0 until cmdLine.length; if (cmdLine(i).equals(argName))) yield i + 1).headOption + val valueIdx = (for (i <- cmdLine.indices; if cmdLine(i).equals(argName)) yield i + 1).headOption valueIdx.filter(i => i < cmdLine.length).map(i => cmdLine(i)) } -} + /** * Embedded config for passing around information of ip and port of created H2O instance. @@ -245,8 +227,8 @@ private class SparklingWaterConfig(val flatfileBVariable: Accumulable[mutable.Ha val sparkHostname: Option[String]) extends AbstractEmbeddedH2OConfig with org.apache.spark.Logging { - /** String containing a flatfile string filled asynchroniously by different thread. */ - @volatile var flatFile:Option[String] = None + /** String containing a flatfile string filled asynchronously by different thread. */ + @volatile var flatFile:Option[String] = None override def notifyAboutEmbeddedWebServerIpPort(ip: InetAddress, port: Int): Unit = { val env = SparkEnv.get @@ -256,48 +238,36 @@ private class SparklingWaterConfig(val flatfileBVariable: Accumulable[mutable.Ha flatfileBVariable.synchronized { flatfileBVariable += thisNodeInfo flatfileBVariable.notifyAll() - } - } - override def notifyAboutCloudSize(ip: InetAddress, port: Int, size: Int): Unit = { - /* do nothing */ - } - - override def fetchFlatfile(): String = { - this.synchronized { while (flatFile.isEmpty) this.wait() } - - flatFile.get + } } - override def providesFlatfile(): Boolean = true + override def notifyAboutCloudSize(ip: InetAddress, port: Int, size: Int): Unit = { + /* do nothing */ + } - override def exit(status: Int): Unit = { /* do nothing */ } + override def fetchFlatfile(): String = { + this.synchronized { while (flatFile.isEmpty) this.wait() } - override def print(): Unit = logInfo("""Debug info: NA""") -} + flatFile.get + } -/** - * Spark environment listener to kill H2O cloud - * if number of Spark block managers change. - */ -private[h2o] -trait SparkEnvListener extends org.apache.spark.scheduler.SparkListener { self: H2OContext => + override def providesFlatfile(): Boolean = true - override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded): Unit = { - println("--------------------> onBlockManagerAdded: " + blockManagerAdded) - } + override def exit(status: Int): Unit = { /* do nothing */ } - override def onBlockManagerRemoved(blockManagerRemoved: SparkListenerBlockManagerRemoved): Unit = { - println("--------------------> onBlockManagerRemoved: " + blockManagerRemoved) + override def print(): Unit = logInfo("""Debug info: NA""") } } +private[internal] object InternalBackendUtils extends InternalBackendUtils + // JVM private H2O is fully initialized signal. // Ugly, but what we can do with h2o private object H2OStartedSignal { @volatile private var started = false def isStarted = started - def setStarted: Unit = { + def setStarted(): Unit = { started = true } } diff --git a/core/src/main/scala/org/apache/spark/h2o/backends/internal/InternalH2OBackend.scala b/core/src/main/scala/org/apache/spark/h2o/backends/internal/InternalH2OBackend.scala new file mode 100644 index 000000000..f902ad6bd --- /dev/null +++ b/core/src/main/scala/org/apache/spark/h2o/backends/internal/InternalH2OBackend.scala @@ -0,0 +1,127 @@ +/* +* 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.h2o.backends.internal + +import org.apache.spark.Logging +import org.apache.spark.h2o.backends.SparklingBackend +import org.apache.spark.h2o.utils.NodeDesc +import org.apache.spark.h2o.{H2OConf, H2OContext} +import org.apache.spark.scheduler.{SparkListener, SparkListenerExecutorAdded} +import water.api.RestAPIManager +import water.{H2O, H2OStarter} + +import scala.util.Random + + +class InternalH2OBackend(@transient val hc: H2OContext) extends SparklingBackend with InternalBackendUtils with Logging{ + + override def stop(stopSparkContext: Boolean): Unit = { + if (stopSparkContext) hc.sparkContext.stop() + H2O.orderlyShutdown(1000) + H2O.exit(0) + } + + override def checkAndUpdateConf(conf: H2OConf): H2OConf = { + super.checkAndUpdateConf(conf) + // Note: updating Spark Conf is useless at this time in more of the cases since SparkContext is already running + + // If 'spark.executor.instances' is specified update H2O property as well + conf.getOption("spark.executor.instances").foreach(v => conf.set("spark.ext.h2o.cluster.size", v)) + + if (!conf.contains("spark.scheduler.minRegisteredResourcesRatio") && !hc.sparkContext.isLocal) { + logWarning("The property 'spark.scheduler.minRegisteredResourcesRatio' is not specified!\n" + + "We recommend to pass `--conf spark.scheduler.minRegisteredResourcesRatio=1`") + // Setup the property but at this point it does not make good sense + conf.set("spark.scheduler.minRegisteredResourcesRatio", "1") + } + + + // Setup properties for H2O configuration + if (conf.cloudName.isEmpty) { + conf.setCloudName("sparkling-water-" + System.getProperty("user.name", "cluster") + "_" + Random.nextInt()) + } + + checkUnsupportedSparkOptions(InternalH2OBackend.UNSUPPORTED_SPARK_OPTIONS, conf) + conf + } + + /** Initialize Sparkling H2O and start H2O cloud. */ + override def init(): Array[NodeDesc] = { + + logInfo(s"Starting H2O services: " + hc.getConf.toString) + // Create dummy RDD distributed over executors + val (spreadRDD, spreadRDDNodes) = new SpreadRDDBuilder(hc, InternalBackendUtils.guessTotalExecutorSize(hc.sparkContext)).build() + + if(hc.getConf.isClusterTopologyListenerEnabled){ + // Attach listener which kills H2O cluster when new Spark executor has been launched ( which means + // that this executors hasn't been discovered during the spreadRDD phase) + hc.sparkContext.addSparkListener(new SparkListener(){ + override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = { + throw new IllegalArgumentException("Executor without H2O instance discovered, killing the cloud!") + } + }) + } + + // Start H2O nodes + // Get executors to execute H2O + val allExecutorIds = spreadRDDNodes.map(_.nodeId).distinct + val executorIds = allExecutorIds + // The collected executors based on IDs should match + assert(spreadRDDNodes.length == executorIds.length, + s"Unexpected number of executors ${spreadRDDNodes.length}!=${executorIds.length}") + // H2O is executed only on the subset of Spark cluster - fail + if (executorIds.length < allExecutorIds.length) { + throw new IllegalArgumentException(s"""Spark cluster contains ${allExecutorIds.length}, + but H2O is running only on ${executorIds.length} nodes!""") + } + // Execute H2O on given nodes + logInfo(s"""Launching H2O on following ${spreadRDDNodes.length} nodes: ${spreadRDDNodes.mkString(",")}""") + + var h2oNodeArgs = InternalBackendUtils.getH2ONodeArgs(hc.getConf) + // Disable web on h2o nodes in non-local mode + if(!hc.sparkContext.isLocal){ + h2oNodeArgs = h2oNodeArgs ++ Array("-disable_web") + } + logDebug(s"Arguments used for launching h2o nodes: ${h2oNodeArgs.mkString(" ")}") + val executors = InternalBackendUtils.startH2O(hc.sparkContext, spreadRDD, spreadRDDNodes.length, h2oNodeArgs, hc.getConf.nodeNetworkMask) + + // Connect to a cluster via H2O client, but only in non-local case + if (!hc.sparkContext.isLocal) { + logTrace("Sparkling H2O - DISTRIBUTED mode: Waiting for " + executors.length) + // Get arguments for this launch including flatfile ( Do not use IP if network mask is specified) + val h2oClientArgs = InternalBackendUtils.toH2OArgs(InternalBackendUtils.getH2OClientArgs(hc.getConf), hc.getConf, executors) + logDebug(s"Arguments used for launching h2o client node: ${h2oClientArgs.mkString(" ")}") + // Launch H2O + H2OStarter.start(h2oClientArgs, false) + } + // And wait for right cluster size + H2O.waitForCloudSize(executors.length, hc.getConf.cloudTimeout) + + // Register web API for client + RestAPIManager.registerClientWebAPI(hc) + H2O.finalizeRegistration() + executors + } + +} + +object InternalH2OBackend{ + val UNSUPPORTED_SPARK_OPTIONS = Seq( + ("spark.dynamicAllocation.enabled", "true"), + ("spark.speculation", "true")) +} diff --git a/core/src/main/scala/org/apache/spark/h2o/backends/internal/InternalReadConverterContext.scala b/core/src/main/scala/org/apache/spark/h2o/backends/internal/InternalReadConverterContext.scala new file mode 100644 index 000000000..2fc8ae4e3 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/h2o/backends/internal/InternalReadConverterContext.scala @@ -0,0 +1,62 @@ +/* +* 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.h2o.backends.internal + +import java.util.UUID + +import org.apache.spark.h2o.converters.ReadConverterContext +import water.fvec.{Chunk, Frame} +import water.{DKV, Key} + +class InternalReadConverterContext(override val keyName: String, override val chunkIdx: Int) extends ReadConverterContext{ + + /** Lazily fetched H2OFrame from K/V store */ + private lazy val fr: Frame = underlyingFrame + + /** Chunks for this partition */ + private lazy val chks: Array[Chunk] = water.fvec.FrameUtils.getChunks(fr, chunkIdx) + + /** Number of rows in this partition */ + private lazy val nrows = chks(0)._len + + override def isNA(columnNum: Int): Boolean = chks(columnNum).isNA(rowIdx) + + override def getLong(columnNum: Int): Long = chks(columnNum).at8(rowIdx) + + override def getDouble(columnNum: Int): Double = chks(columnNum).atd(rowIdx) + + override def getString(columnNum: Int): String = { + if (chks(columnNum).vec().isCategorical) { + val str = chks(columnNum).vec().domain()(chks(columnNum).at8(rowIdx).asInstanceOf[Int]) + str + } else if (chks(columnNum).vec().isString) { + chks(columnNum).atStr(valStr, rowIdx) // TODO improve this. + valStr.toString + } else if (chks(columnNum).vec().isUUID) { + val uuid = new UUID(chks(columnNum).at16h(rowIdx), chks(columnNum).at16l(rowIdx)) + uuid.toString + } else{ + assert(assertion = false, "Should never be here") + null + } + } + + private def underlyingFrame = DKV.get(Key.make(keyName)).get.asInstanceOf[Frame] + + override def numRows: Int = nrows +} diff --git a/core/src/main/scala/org/apache/spark/h2o/backends/internal/InternalWriteConverterContext.scala b/core/src/main/scala/org/apache/spark/h2o/backends/internal/InternalWriteConverterContext.scala new file mode 100644 index 000000000..adb2bcce0 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/h2o/backends/internal/InternalWriteConverterContext.scala @@ -0,0 +1,48 @@ +/* +* 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.h2o.backends.internal + +import java.sql.Timestamp + +import org.apache.spark.h2o.converters.WriteConverterContext +import water.fvec.{FrameUtils, NewChunk} + +class InternalWriteConverterContext extends WriteConverterContext{ + var nchnks: Array[NewChunk] = _ + override def createChunks(keyName: String, vecTypes: Array[Byte], chunkId: Int): Unit = { + nchnks = FrameUtils.createNewChunks(keyName, vecTypes, chunkId) + } + + override def closeChunks(): Unit = { + FrameUtils.closeNewChunks(nchnks) + } + + override def put(columnNum: Int, n: Number): Unit = nchnks(columnNum).addNum(n.doubleValue()) + + override def put(columnNum: Int, n: Boolean): Unit = nchnks(columnNum).addNum(if (n) 1 else 0) + + override def put(columnNum: Int, n: Timestamp): Unit = nchnks(columnNum).addNum(n.getTime) + + override def put(columnNum: Int, str: String): Unit = nchnks(columnNum).addStr(str) + + override def putNA(columnNum: Int): Unit = nchnks(columnNum).addNA() + + override def increaseRowCounter(): Unit = { }// empty on purpose, we can get number or rows without counter + + override def numOfRows: Long = nchnks(0).len() +} diff --git a/core/src/main/scala/org/apache/spark/h2o/InvokeOnNodesRDD.scala b/core/src/main/scala/org/apache/spark/h2o/backends/internal/InvokeOnNodesRDD.scala similarity index 90% rename from core/src/main/scala/org/apache/spark/h2o/InvokeOnNodesRDD.scala rename to core/src/main/scala/org/apache/spark/h2o/backends/internal/InvokeOnNodesRDD.scala index 2bae8cc78..84ed31965 100644 --- a/core/src/main/scala/org/apache/spark/h2o/InvokeOnNodesRDD.scala +++ b/core/src/main/scala/org/apache/spark/h2o/backends/internal/InvokeOnNodesRDD.scala @@ -15,11 +15,12 @@ * limitations under the License. */ -package org.apache.spark.h2o +package org.apache.spark.h2o.backends.internal -import org.apache.spark.{TaskContext, Partition, SparkContext} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.h2o.H2OContextUtils.NodeDesc +import org.apache.spark.h2o.utils.NodeDesc +import org.apache.spark.rdd.RDD +import org.apache.spark.{Partition, SparkContext, TaskContext} /** Special kind of RDD which is used to invoke code on all executors detected in cluster. */ private[h2o] diff --git a/core/src/main/scala/org/apache/spark/h2o/SpreadRDDBuilder.scala b/core/src/main/scala/org/apache/spark/h2o/backends/internal/SpreadRDDBuilder.scala similarity index 67% rename from core/src/main/scala/org/apache/spark/h2o/SpreadRDDBuilder.scala rename to core/src/main/scala/org/apache/spark/h2o/backends/internal/SpreadRDDBuilder.scala index b4c0d202b..3e82cd841 100644 --- a/core/src/main/scala/org/apache/spark/h2o/SpreadRDDBuilder.scala +++ b/core/src/main/scala/org/apache/spark/h2o/backends/internal/SpreadRDDBuilder.scala @@ -15,10 +15,12 @@ * limitations under the License. */ -package org.apache.spark.h2o +package org.apache.spark.h2o.backends.internal -import org.apache.spark.{SparkEnv, SparkContext} -import org.apache.spark.h2o.H2OContextUtils._ +import org.apache.spark.SparkEnv +import org.apache.spark.h2o.backends.SharedBackendUtils +import org.apache.spark.h2o.utils.NodeDesc +import org.apache.spark.h2o.{H2OContext, RDD} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.local.LocalBackend @@ -28,16 +30,15 @@ import scala.annotation.tailrec * An H2O specific builder for InvokeOnNodesRDD. */ private[spark] -class SpreadRDDBuilder(sc: SparkContext, - numExecutorHint: Option[Int] = None) extends { - val sparkConf = sc.getConf - } with H2OConf with org.apache.spark.Logging { - - val numExecutors = numH2OWorkers +class SpreadRDDBuilder(hc: H2OContext, + numExecutorHint: Option[Int] = None) extends org.apache.spark.Logging with SharedBackendUtils { + private val conf = hc.getConf + private val sc = hc.sparkContext + private val numExecutors = conf.numH2OWorkers def build(): (RDD[NodeDesc], Array[NodeDesc]) = { logDebug(s"Building SpreadRDD: numExecutors=${numExecutors}, numExecutorHint=${numExecutorHint}") - build(numRddRetries, drddMulFactor, 0) + build(conf.numRddRetries, conf.drddMulFactor, 0) } @tailrec @@ -46,17 +47,17 @@ class SpreadRDDBuilder(sc: SparkContext, mfactor: Int, numTriesSame: Int): (RDD[NodeDesc], Array[NodeDesc]) = { logDebug(s"Creating RDD for launching H2O nodes (mretries=${nretries}, mfactor=${mfactor}, " + - s"numTriesSame=${numTriesSame}, backend#isReady=${isBackendReady()}") + s"numTriesSame=${numTriesSame}, backend#isReady=${isBackendReady()}") // Get number of available Spark executors, invoke distributed operation and compute // number of visible nodes again val nSparkExecBefore = numOfSparkExecutors // Number of expected workers - val expectedWorkers = numExecutors.orElse(numExecutorHint).getOrElse(if (nSparkExecBefore > 0) nSparkExecBefore else defaultCloudSize) + val expectedWorkers = numExecutors.orElse(numExecutorHint).getOrElse(if (nSparkExecBefore > 0) nSparkExecBefore else conf.defaultCloudSize) // Create some distributed data val spreadRDD = sc.parallelize(0 until mfactor*expectedWorkers, mfactor*expectedWorkers + 1).persist() // Collect information about executors in Spark cluster val visibleNodes = collectNodesInfo(spreadRDD) - val numVisibleNodes = visibleNodes.map(_.executorId).distinct.length + val numVisibleNodes = visibleNodes.map(_.nodeId).distinct.length // Number of Spark executors after distributed operation val nSparkExecAfter = numOfSparkExecutors // Delete RDD @@ -64,27 +65,27 @@ class SpreadRDDBuilder(sc: SparkContext, // Decide about visible state if ((numVisibleNodes < expectedWorkers || nSparkExecAfter != nSparkExecBefore) - && nretries == 0) { + && nretries == 0) { // We tried many times, but we were not able to get right number of executors throw new IllegalArgumentException( s"""Cannot execute H2O on all Spark executors: - | Expected number of H2O workers is $numExecutorHint - | Detected number of Spark workers is $numVisibleNodes - | Num of Spark executors before is $nSparkExecBefore - | Num of Spark executors after is $nSparkExecAfter + | Expected number of H2O workers is ${numExecutorHint} + | Detected number of Spark workers is ${numVisibleNodes} + | Num of Spark executors before is ${nSparkExecBefore} + | Num of Spark executors after is ${nSparkExecAfter} |""".stripMargin ) } else if (nSparkExecAfter != nSparkExecBefore || nSparkExecAfter != numVisibleNodes) { // We detected change in number of executors - logInfo(s"Detected $nSparkExecBefore before, and $nSparkExecAfter spark executors after, backend#isReady=${isBackendReady()}! Retrying again...") + logInfo(s"Detected ${nSparkExecBefore} before, and ${nSparkExecAfter} spark executors after, backend#isReady=${isBackendReady()}! Retrying again...") build(nretries - 1, 2*mfactor, 0) - } else if (( numTriesSame == subseqTries) - || (numExecutors.isEmpty && numVisibleNodes == expectedWorkers) - || (numExecutors.isDefined && numExecutors.get == numVisibleNodes)) { - logInfo(s"Detected $numVisibleNodes spark executors for $expectedWorkers H2O workers!") + } else if ((numTriesSame == conf.subseqTries) + || (numExecutors.isEmpty && numVisibleNodes == expectedWorkers) + || (numExecutors.isDefined && numExecutors.get == numVisibleNodes)) { + logInfo(s"Detected ${numVisibleNodes} spark executors for ${expectedWorkers} H2O workers!") (new InvokeOnNodesRDD(visibleNodes, sc), visibleNodes) } else { - logInfo(s"Detected $numVisibleNodes spark executors for $expectedWorkers H2O workers, backend#isReady=${isBackendReady()}! Retrying again...") + logInfo(s"Detected ${numVisibleNodes} spark executors for ${expectedWorkers} H2O workers, backend#isReady=${isBackendReady()}! Retrying again...") build(nretries-1, mfactor, numTriesSame + 1) } } @@ -98,11 +99,10 @@ class SpreadRDDBuilder(sc: SparkContext, // Collect flatfile - tuple of (executorId, IP, -1) val nodes = distRDD.mapPartitionsWithIndex { (idx, it) => val env = SparkEnv.get - Iterator.single(NodeDesc(env.executorId, getHostname(env), -1) - ) + Iterator.single(NodeDesc(env.executorId, SharedBackendUtils.getHostname(env), -1)) }.collect() // Take only unique executors - nodes.groupBy(_.executorId).map(_._2.head).toArray.sortWith(_.executorId < _.executorId) + nodes.groupBy(_.nodeId).map(_._2.head).toArray.sortWith(_.nodeId < _.nodeId) } /** diff --git a/core/src/main/scala/org/apache/spark/h2o/converters/ConverterUtils.scala b/core/src/main/scala/org/apache/spark/h2o/converters/ConverterUtils.scala new file mode 100644 index 000000000..36038147b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/h2o/converters/ConverterUtils.scala @@ -0,0 +1,133 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.h2o.converters + +import org.apache.spark.TaskContext +import org.apache.spark.h2o._ +import org.apache.spark.h2o.backends.internal.{InternalReadConverterContext, InternalWriteConverterContext} +import org.apache.spark.h2o.utils.NodeDesc +import water.{DKV, Key} + +import scala.collection.immutable +import scala.collection.mutable.ListBuffer +import scala.reflect.runtime.universe._ + + +private[converters] trait ConverterUtils { + + + def initFrame[T](keyName: String, names: Array[String]):Unit = { + val fr = new water.fvec.Frame(Key.make(keyName)) + water.fvec.FrameUtils.preparePartialFrame(fr, names) + // Save it directly to DKV + fr.update() + } + + + def finalizeFrame[T](keyName: String, + res: Array[Long], + colTypes: Array[Byte], + colDomains: Array[Array[String]] = null):Frame = { + val fr:Frame = DKV.get(keyName).get.asInstanceOf[Frame] + water.fvec.FrameUtils.finalizePartialFrame(fr, res, colDomains, colTypes) + fr + } + + /** + * Gets frame for specified key or none if that frame does not exist + * + * @param keyName key of the requested frame + * @return option containing frame or none + */ + def getFrameOrNone(keyName: String): Option[H2OFrame] = { + // Fetch cached frame from DKV + val frameVal = DKV.get(keyName) + + if(frameVal != null){ + Some(new H2OFrame(frameVal.get.asInstanceOf[Frame])) + }else{ + None + } + } + + /** + * Converts the RDD to H2O Frame using specified conversion function + * + * @param hc H2O context + * @param rdd rdd to convert + * @param keyName key of the resulting frame + * @param colNames names of the columns in the H2O Frame + * @param vecTypes types of the vectors in the H2O Frame + * @param func conversion function - the function takes parameters needed extra by specific transformations + * and returns function which does the general transformation + * @tparam T type of RDD to convert + * @return H2O Frame + */ + def convert[T](hc: H2OContext, rdd : RDD[T], keyName: String, colNames: Array[String], vecTypes: Array[Byte], + func: ( (String, Array[Byte], Option[immutable.Map[Int, NodeDesc]]) => (TaskContext, Iterator[T]) => (Int, Long))) = { + // Make an H2O data Frame - but with no backing data (yet) + initFrame(keyName, colNames) + + // prepare rdd and required metadata based on the used backend + val (preparedRDD, uploadPlan) = (rdd, None) + + val rows = hc.sparkContext.runJob(preparedRDD, func(keyName, vecTypes, uploadPlan)) // eager, not lazy, evaluation + val res = new Array[Long](preparedRDD.partitions.length) + rows.foreach { case (cidx, nrows) => res(cidx) = nrows } + // Add Vec headers per-Chunk, and finalize the H2O Frame + new H2OFrame(finalizeFrame(keyName, res, vecTypes)) + } +} + +object ConverterUtils { + def getWriteConverterContext(uploadPlan: Option[immutable.Map[Int, NodeDesc]], + partitionId: Int): WriteConverterContext = { + val converterContext = new InternalWriteConverterContext() + converterContext + } + + def getReadConverterContext(isExternalBackend: Boolean, + keyName: String, + chksLocation: Option[Array[NodeDesc]], + types: Option[Array[Byte]], + chunkIdx: Int): ReadConverterContext = { + val converterContext = new InternalReadConverterContext(keyName, chunkIdx) + converterContext + } + + def getIterator[T](isExternalBackend: Boolean, + iterator: Iterator[T]): Iterator[T] = { + if (isExternalBackend) { + val rows = new ListBuffer[T]() + while (iterator.hasNext) { + rows += iterator.next() + } + rows.iterator + } else { + iterator + } + } + + def prepareExpectedTypes[T: TypeTag](isExternalBackend: Boolean, + types: Array[T]): Option[Array[Byte]] = { + // For now return None because internal backend is used at all cases and we don't need any additional info at this time. + None + } + +} + diff --git a/core/src/main/scala/org/apache/spark/h2o/converters/H2ODataFrame.scala b/core/src/main/scala/org/apache/spark/h2o/converters/H2ODataFrame.scala new file mode 100644 index 000000000..541ea8ae9 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/h2o/converters/H2ODataFrame.scala @@ -0,0 +1,105 @@ +/* +* 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.h2o.converters + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.h2o._ +import org.apache.spark.h2o.utils.H2OSchemaUtils._ +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.GenericMutableRow +import org.apache.spark.sql.types._ +import org.apache.spark.{Partition, SparkContext, TaskContext} + +/** + * H2O H2OFrame wrapper providing RDD[Row]=DataFrame API. + * + * @param frame frame which will be wrapped as DataFrame + * @param requiredColumns list of the columns which should be provided by iterator, null means all + * @param sc an instance of Spark context + */ +private[spark] +class H2ODataFrame[T <: water.fvec.Frame](@transient val frame: T, + val requiredColumns: Array[String]) + (@transient val sc: SparkContext) + extends RDD[InternalRow](sc, Nil) with H2ORDDLike[T] { + + def this(@transient frame: T) + (@transient sc: SparkContext) = this(frame, null)(sc) + + override val isExternalBackend = H2OConf(sc).runsInExternalClusterMode + + @DeveloperApi + override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { + // Prepare iterator + val iterator = new H2OChunkIterator[InternalRow] { + /** Frame reference */ + override val keyName = frameKeyName + /** Processed partition index */ + override val partIndex = split.index + /** Selected column indices */ + val selectedColumnIndices = if (requiredColumns == null) { + fr.names().indices + } else { + requiredColumns.toSeq.map { name => + fr.find(name) + } + } + // Make sure that column selection is consistent + // scalastyle:off + assert(requiredColumns != null && selectedColumnIndices.length == requiredColumns.length, + "Column selection missing a column!") + // scalastyle:on + + /** Types used for data transfer */ + val types = selectedColumnIndices.map(idx => vecTypeToDataType(fr.vec(idx))).toArray + override val expectedTypes: Option[Array[Byte]] = ConverterUtils.prepareExpectedTypes(isExternalBackend, types) + + override def next(): InternalRow = { + /** Mutable reusable row returned by iterator */ + val mutableRow = new GenericMutableRow(selectedColumnIndices.length) + selectedColumnIndices.indices.foreach { idx => + val i = selectedColumnIndices(idx) + val typ = types(idx) + if (converterCtx.isNA(i)) { + mutableRow.setNullAt(idx) + } else { + typ match { + case ByteType => mutableRow.setByte(idx, converterCtx.getByte(i)) + case ShortType => mutableRow.setShort(idx, converterCtx.getShort(i)) + case IntegerType => mutableRow.setInt(idx, converterCtx.getInt(i)) + case LongType => mutableRow.setLong(idx, converterCtx.getLong(i)) + case FloatType => mutableRow.setFloat(idx, converterCtx.getFloat(i)) + case DoubleType => mutableRow.setDouble(idx, converterCtx.getDouble(i)) + case BooleanType => mutableRow.setBoolean(idx, converterCtx.getBoolean(i)) + case StringType => mutableRow.update(idx, converterCtx.getUTF8String(i)) + case TimestampType => mutableRow.setLong(idx, converterCtx.getTimestamp(i)) + case _ => ??? + } + } + } + converterCtx.increaseRowIdx() + // Return result + mutableRow + } + } + + // Wrap the iterator to backend specifc wrapper + ConverterUtils.getIterator[InternalRow](isExternalBackend, iterator) + } +} diff --git a/core/src/main/scala/org/apache/spark/rdd/H2ORDD.scala b/core/src/main/scala/org/apache/spark/h2o/converters/H2ORDD.scala similarity index 54% rename from core/src/main/scala/org/apache/spark/rdd/H2ORDD.scala rename to core/src/main/scala/org/apache/spark/h2o/converters/H2ORDD.scala index a1e2e8457..fd2c058c6 100644 --- a/core/src/main/scala/org/apache/spark/rdd/H2ORDD.scala +++ b/core/src/main/scala/org/apache/spark/h2o/converters/H2ORDD.scala @@ -15,30 +15,35 @@ * limitations under the License. */ -package org.apache.spark.rdd +package org.apache.spark.h2o.converters -import org.apache.spark.h2o.{H2OContext, H2OFrame, ReflectionUtils} +import org.apache.spark.h2o.{H2OConf, H2OContext} +import org.apache.spark.h2o.utils.ReflectionUtils +import org.apache.spark.rdd.RDD import org.apache.spark.{Partition, SparkContext, TaskContext} import water.fvec.Frame -import water.parser.BufferedString import scala.reflect.ClassTag import scala.reflect.runtime.universe._ /** - * Convert H2OFrame into an RDD (lazily) - */ - + * Convert H2OFrame into an RDD (lazily). + * @param frame an instance of H2O frame + * @param colNames names of columns + * @param sc an instance of Spark context + * @tparam A type for resulting RDD + * @tparam T specific type of H2O frame + */ private[spark] class H2ORDD[A <: Product: TypeTag: ClassTag, T <: Frame] private(@transient val frame: T, val colNames: Array[String]) - (@transient sparkContext: SparkContext) - extends RDD[A](sparkContext, Nil) with H2ORDDLike[T] { + (@transient sc: SparkContext) + extends RDD[A](sc, Nil) with H2ORDDLike[T] { // Get column names before building an RDD def this(@transient fr : T) - (@transient sparkContext: SparkContext) = this(fr, ReflectionUtils.names[A])(sparkContext) + (@transient sc: SparkContext) = this(fr, ReflectionUtils.names[A])(sc) // Check that H2OFrame & given Scala type are compatible if (colNames.length > 1) { @@ -49,58 +54,55 @@ class H2ORDD[A <: Product: TypeTag: ClassTag, T <: Frame] private(@transient val } } } + val types = ReflectionUtils.types[A](colNames) + override val isExternalBackend = H2OConf(sc).runsInExternalClusterMode /** * :: DeveloperApi :: * Implemented by subclasses to compute a given partition. */ override def compute(split: Partition, context: TaskContext): Iterator[A] = { - val kn = frameKeyName - new H2OChunkIterator[A] { - override val keyName = kn - override val partIndex = split.index + + val iterator = new H2OChunkIterator[A] { val jc = implicitly[ClassTag[A]].runtimeClass val cs = jc.getConstructors val ccr = cs.collectFirst({ - case c if (c.getParameterTypes.length==colNames.length) => c + case c if c.getParameterTypes.length == colNames.length => c }) .getOrElse({ throw new IllegalArgumentException( s"Constructor must take exactly ${colNames.length} args") }) - /** Dummy muttable holder for String values */ - val valStr = new BufferedString() + + override val expectedTypes: Option[Array[Byte]] = ConverterUtils.prepareExpectedTypes(isExternalBackend, types) + override val keyName = frameKeyName + override val partIndex = split.index def next(): A = { - val data = new Array[Option[Any]](chks.length) - for ( - idx <- 0 until chks.length; - chk = chks (idx); - typ = types(idx) - ) { - val value = if (chk.isNA(row)) None - else typ match { - case q if q == classOf[Integer] => Some(chk.at8(row).asInstanceOf[Int]) - case q if q == classOf[java.lang.Long] => Some(chk.at8(row)) - case q if q == classOf[java.lang.Double] => Some(chk.atd(row)) - case q if q == classOf[java.lang.Float] => Some(chk.atd(row)) - case q if q == classOf[java.lang.Boolean] => Some(chk.at8(row) == 1) - case q if q == classOf[String] => - if (chk.vec().isCategorical) { - Some(chk.vec().domain()(chk.at8(row).asInstanceOf[Int])) - } else if (chk.vec().isString) { - chk.atStr(valStr, row) - Some(valStr.toString) - } else None - case _ => None - } - data(idx) = value + val data = new Array[Option[Any]](ncols) + // FIXME: this is not perfect since ncols does not need to match number of names + (0 until ncols).foreach{ idx => + val value = if (converterCtx.isNA(idx)) None + else types(idx) match { + case q if q == classOf[Integer] => Some(converterCtx.getInt(idx)) + case q if q == classOf[java.lang.Long] => Some(converterCtx.getLong(idx)) + case q if q == classOf[java.lang.Double] => Some(converterCtx.getDouble(idx)) + case q if q == classOf[java.lang.Float] => Some(converterCtx.getFloat(idx)) + case q if q == classOf[java.lang.Boolean] => Some(converterCtx.getBoolean(idx)) + case q if q == classOf[String] => Option(converterCtx.getString(idx)) + case _ => None } - row += 1 + data(idx) = value + } + + converterCtx.increaseRowIdx() + // Create instance for the extracted row ccr.newInstance(data:_*).asInstanceOf[A] } } + + ConverterUtils.getIterator[A](isExternalBackend, iterator) } } diff --git a/core/src/main/scala/org/apache/spark/rdd/H2ORDDLike.scala b/core/src/main/scala/org/apache/spark/h2o/converters/H2ORDDLike.scala similarity index 61% rename from core/src/main/scala/org/apache/spark/rdd/H2ORDDLike.scala rename to core/src/main/scala/org/apache/spark/h2o/converters/H2ORDDLike.scala index 952082e5d..907517a97 100644 --- a/core/src/main/scala/org/apache/spark/rdd/H2ORDDLike.scala +++ b/core/src/main/scala/org/apache/spark/h2o/converters/H2ORDDLike.scala @@ -15,16 +15,17 @@ * limitations under the License. */ -package org.apache.spark.rdd +package org.apache.spark.h2o.converters import org.apache.spark.Partition -import water.fvec.{Chunk, Frame} +import water.fvec.{Frame, FrameUtils} import water.{DKV, Key} +import scala.reflect.runtime.universe._ /** * Contains functions that are shared between all H2ORDD types (i.e., Scala, Java) */ -private[rdd] trait H2ORDDLike[T <: Frame] { +private[converters] trait H2ORDDLike[T <: Frame] { /** Underlying DataFrame */ @transient val frame: T @@ -34,6 +35,12 @@ private[rdd] trait H2ORDDLike[T <: Frame] { /** Number of chunks per a vector */ val numChunks: Int = frame.anyVec().nChunks() + /** Is the external backend in use */ + val isExternalBackend: Boolean + + /** Chunk locations helps us to determine the node which really has the data we needs. */ + val chksLocation = if (isExternalBackend) Some(FrameUtils.getChunksLocations(frame)) else None + protected def getPartitions: Array[Partition] = { val res = new Array[Partition](numChunks) for(i <- 0 until numChunks) res(i) = new Partition { val index = i } @@ -42,23 +49,29 @@ private[rdd] trait H2ORDDLike[T <: Frame] { /** Base implementation for iterator over rows stored in chunks for given partition. */ trait H2OChunkIterator[+A] extends Iterator[A] { + /* Key of pointing to underlying dataframe */ val keyName: String /* Partition index */ val partIndex: Int /* Lazily fetched dataframe from K/V store */ lazy val fr: Frame = getFrame() - /* Chunks for this partition */ - lazy val chks: Array[Chunk] = water.fvec.FrameUtils.getChunks(fr, partIndex) - /* Number of rows in this partition */ - lazy val nrows = chks(0)._len - /* Number of columns in the dataset */ + /* Number of columns in the full dataset */ lazy val ncols = fr.numCols() - /* Iterator state: Actual row */ - var row: Int = 0 + /** Create new types list which describes expected types in a way external H2O backend can use it. This list + * contains types in a format same for H2ODataFrame and H2ORDD */ + val expectedTypes: Option[Array[Byte]] + + /* Converter context */ + lazy val converterCtx: ReadConverterContext = + ConverterUtils.getReadConverterContext(isExternalBackend, + keyName, + chksLocation, + expectedTypes, + partIndex) - def hasNext: Boolean = row < nrows + override def hasNext: Boolean = converterCtx.hasNext private def getFrame() = DKV.get(Key.make(keyName)).get.asInstanceOf[Frame] } diff --git a/core/src/main/scala/org/apache/spark/h2o/converters/LabeledPointConverter.scala b/core/src/main/scala/org/apache/spark/h2o/converters/LabeledPointConverter.scala new file mode 100644 index 000000000..74c0dac61 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/h2o/converters/LabeledPointConverter.scala @@ -0,0 +1,104 @@ +/* +* 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.h2o.converters + +import org.apache.spark.h2o._ +import org.apache.spark.h2o.utils.{H2OTypeUtils, NodeDesc, ReflectionUtils} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.{Logging, TaskContext} +import water.Key +import water.fvec.H2OFrame + +import scala.collection.immutable +import scala.language.implicitConversions +import scala.reflect.runtime.universe._ + +private[converters] object LabeledPointConverter extends Logging with ConverterUtils{ + + /** Transform RDD[LabeledPoint] to appropriate H2OFrame */ + def toH2OFrame(hc: H2OContext, rdd: RDD[LabeledPoint], frameKeyName: Option[String]): H2OFrame = { + import H2OTypeUtils._ + import ReflectionUtils._ + + val keyName = frameKeyName.getOrElse("frame_rdd_" + rdd.id + Key.rand()) + + // first convert vector to dense vector + val rddDense = rdd.map(labeledPoint => new LabeledPoint(labeledPoint.label,labeledPoint.features.toDense)) + val numFeatures = rddDense.map(labeledPoint => labeledPoint.features.size) + val maxNumFeatures = numFeatures.max() + val minNumFeatures = numFeatures.min() + if(minNumFeatures "feature" + num).toSeq).toArray[String] + val ftypes = 0.until(maxNumFeatures + 1).map(_ => typ(typeOf[Double])) + val vecTypes = ftypes.indices.map(idx => dataTypeToVecType(ftypes(idx))).toArray + + convert[LabeledPoint](hc, rdd, keyName, fnames, vecTypes, perLabeledPointRDDPartition(maxNumFeatures)) + } + + /** + * + * @param keyName key of the frame + * @param vecTypes h2o vec types + * @param maxNumFeatures maximum number of features in the labeled point + * @param uploadPlan plan which assigns each partition h2o node where the data from that partition will be uploaded + * @param context spark task context + * @param it iterator over data in the partition + * @return pair (partition ID, number of rows in this partition) + */ + private[this] + def perLabeledPointRDDPartition(maxNumFeatures: Int) + (keyName: String, vecTypes: Array[Byte], uploadPlan: Option[immutable.Map[Int, NodeDesc]]) + (context: TaskContext, it: Iterator[LabeledPoint]): (Int, Long) = { + val con = ConverterUtils.getWriteConverterContext(uploadPlan, context.partitionId()) + + // Creates array of H2O NewChunks; A place to record all the data in this partition + con.createChunks(keyName, vecTypes, context.partitionId()) + + it.foreach(labeledPoint => { + // For all LabeledPoints in RDD + var nextChunkId = 0 + + // Add LabeledPoint label + con.put(nextChunkId, labeledPoint.label) + nextChunkId = nextChunkId + 1 + + for( i<-0 until labeledPoint.features.size) { + // For all features... + con.put(nextChunkId, labeledPoint.features(i)) + nextChunkId = nextChunkId + 1 + } + + for( i<-labeledPoint.features.size until maxNumFeatures){ + // Fill missing features with n/a + con.putNA(nextChunkId) + nextChunkId = nextChunkId + 1 + } + + con.increaseRowCounter() + }) + + //Compress & write data in partitions to H2O Chunks + con.closeChunks() + + // Return Partition number and number of rows in this partition + (context.partitionId, con.numOfRows) + } +} diff --git a/core/src/main/scala/org/apache/spark/h2o/converters/PrimitiveRDDConverter.scala b/core/src/main/scala/org/apache/spark/h2o/converters/PrimitiveRDDConverter.scala new file mode 100644 index 000000000..31e5983c0 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/h2o/converters/PrimitiveRDDConverter.scala @@ -0,0 +1,82 @@ +/* +* 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.h2o.converters + +import org.apache.spark.h2o._ +import org.apache.spark.h2o.utils.{H2OTypeUtils, NodeDesc, ReflectionUtils} +import org.apache.spark.{Logging, TaskContext} +import water.Key +import water.fvec.H2OFrame + +import scala.collection.immutable +import scala.language.implicitConversions +import scala.reflect.runtime.universe._ + +private[converters] object PrimitiveRDDConverter extends Logging with ConverterUtils{ + + def toH2OFrame[T: TypeTag](hc: H2OContext, rdd: RDD[T], frameKeyName: Option[String]): H2OFrame = { + import H2OTypeUtils._ + import ReflectionUtils._ + + val keyName = frameKeyName.getOrElse("frame_rdd_" + rdd.id + Key.rand()) + + val fnames = Array[String]("values") + val ftypes = Array[Class[_]](typ(typeOf[T])) + val vecTypes = ftypes.indices.map(idx => dataTypeToVecType(ftypes(idx))).toArray + + convert[T](hc, rdd, keyName, fnames, vecTypes, perPrimitiveRDDPartition()) + } + + + /** + * + * @param keyName key of the frame + * @param vecTypes h2o vec types + * @param uploadPlan if external backend is used, then it is a plan which assigns each partition h2o + * node where the data from that partition will be uploaded, otherwise is Node + * @param context spark task context + * @param it iterator over data in the partition + * @tparam T type of data inside the RDD + * @return pair (partition ID, number of rows in this partition) + */ + private[this] + def perPrimitiveRDDPartition[T]() // extra arguments for this transformation + (keyName: String, vecTypes: Array[Byte], uploadPlan: Option[immutable.Map[Int, NodeDesc]]) // general arguments + (context: TaskContext, it: Iterator[T]): (Int, Long) = { // arguments and return types needed for spark's runJob input + val con = ConverterUtils.getWriteConverterContext(uploadPlan, context.partitionId()) + + con.createChunks(keyName, vecTypes, context.partitionId()) + // try to wait for reply to ensure we can continue with sending + it.foreach { r => + r match { + case n: Number => con.put(0, n) + case n: Boolean => con.put(0, n) + case n: String => con.put(0, n) + case n: java.sql.Timestamp => con.put(0, n) + case _ => con.putNA(0) + } + con.increaseRowCounter() + } + //Compress & write data in partitions to H2O Chunks + con.closeChunks() + + // Return Partition number and number of rows in this partition + (context.partitionId, con.numOfRows) + } + +} diff --git a/core/src/main/scala/org/apache/spark/h2o/converters/ProductRDDConverter.scala b/core/src/main/scala/org/apache/spark/h2o/converters/ProductRDDConverter.scala new file mode 100644 index 000000000..baf0f22f5 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/h2o/converters/ProductRDDConverter.scala @@ -0,0 +1,136 @@ +/* +* 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.h2o.converters + +import org.apache.spark.h2o._ +import org.apache.spark.h2o.utils.H2OTypeUtils._ +import org.apache.spark.h2o.utils.{H2OTypeUtils, NodeDesc, ReflectionUtils} +import org.apache.spark.{Logging, TaskContext} +import water.Key + +import scala.collection.immutable +import scala.collection.mutable.ListBuffer +import scala.language.implicitConversions +import scala.reflect.ClassTag +import scala.reflect.runtime.universe._ + +private[converters] object ProductRDDConverter extends Logging with ConverterUtils{ + + /** Transform H2OFrame to Product RDD */ + def toRDD[A <: Product: TypeTag: ClassTag, T <: Frame](hc: H2OContext, fr: T): RDD[A] = { + new H2ORDD[A, T](fr)(hc.sparkContext) + } + + /** Transform RDD to H2OFrame. This method expects RDD of type Product without TypeTag */ + def toH2OFrame(hc: H2OContext, rdd: RDD[Product], frameKeyName: Option[String]): H2OFrame = { + + val keyName = frameKeyName.getOrElse("frame_rdd_" + rdd.id + Key.rand()) // There are uniq IDs for RDD + + // infer the type + val first = rdd.first() + val fnames = 0.until(first.productArity).map(idx => "f" + idx).toArray[String] + val ftypes = new ListBuffer[Class[_]]() + val it = first.productIterator + while(it.hasNext){ + ftypes+=inferFieldType(it.next()) + } + // Collect H2O vector types for all input types + val vecTypes = ftypes.toArray[Class[_]].indices.map(idx => dataTypeToVecType(ftypes(idx))).toArray + + convert[Product](hc, rdd, keyName, fnames, vecTypes, perTypedRDDPartition()) + } + + /** Transform typed RDD into H2O Frame */ + def toH2OFrame[T <: Product : TypeTag](hc: H2OContext, rdd: RDD[T], frameKeyName: Option[String]) : H2OFrame = { + import H2OTypeUtils._ + import ReflectionUtils._ + + val keyName = frameKeyName.getOrElse("frame_rdd_" + rdd.id + Key.rand()) // There are uniq IDs for RDD + + val fnames = names[T] + val ftypes = types[T](fnames) + // Collect H2O vector types for all input types + val vecTypes = ftypes.indices.map(idx => dataTypeToVecType(ftypes(idx))).toArray + + convert[T](hc, rdd, keyName, fnames, vecTypes, perTypedRDDPartition()) + } + + /** + * + * @param keyName key of the frame + * @param vecTypes h2o vec types + * @param uploadPlan plan which assigns each partition h2o node where the data from that partition will be uploaded + * @param context spark task context + * @param it iterator over data in the partition + * @tparam T type of data inside the RDD + * @return pair (partition ID, number of rows in this partition) + */ + private[this] + def perTypedRDDPartition[T<:Product]() + (keyName: String, vecTypes: Array[Byte], uploadPlan: Option[immutable.Map[Int, NodeDesc]]) + ( context: TaskContext, it: Iterator[T] ): (Int,Long) = { + val con = ConverterUtils.getWriteConverterContext(uploadPlan, context.partitionId()) + // Creates array of H2O NewChunks; A place to record all the data in this partition + con.createChunks(keyName, vecTypes, context.partitionId()) + + it.foreach(prod => { // For all rows which are subtype of Product + for( i <- 0 until prod.productArity ) { // For all fields... + val fld = prod.productElement(i) + val x = fld match { + case Some(n) => n + case _ => fld + } + x match { + case n: Number => con.put(i, n) + case n: Boolean => con.put(i, n) + case n: String => con.put(i, n) + case n : java.sql.Timestamp => con.put(i, n) + case _ => con.putNA(i) + } + } + con.increaseRowCounter() + }) + + //Compress & write data in partitions to H2O Chunks + con.closeChunks() + + // Return Partition number and number of rows in this partition + (context.partitionId, con.numOfRows) + } + + /** + * Infers the type from Any, used for determining the types in Product RDD + * + * @param value + * @return + */ + private[this] def inferFieldType(value : Any): Class[_] ={ + value match { + case n: Byte => classOf[java.lang.Byte] + case n: Short => classOf[java.lang.Short] + case n: Int => classOf[java.lang.Integer] + case n: Long => classOf[java.lang.Long] + case n: Float => classOf[java.lang.Float] + case n: Double => classOf[java.lang.Double] + case n: Boolean => classOf[java.lang.Boolean] + case n: String => classOf[java.lang.String] + case n: java.sql.Timestamp => classOf[java.sql.Timestamp] + case q => throw new IllegalArgumentException(s"Do not understand type $q") + } + } +} diff --git a/core/src/main/scala/org/apache/spark/h2o/converters/ReadConverterContext.scala b/core/src/main/scala/org/apache/spark/h2o/converters/ReadConverterContext.scala new file mode 100644 index 000000000..028e3640c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/h2o/converters/ReadConverterContext.scala @@ -0,0 +1,67 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.h2o.converters + +import org.apache.spark.unsafe.types.UTF8String +import water.parser.BufferedString + + +/** + * Methods which each ReadConverterContext has to implement. + * + * Read Converter Context is a class which holds the state of connection/chunks and allows us to read/download data from those chunks + * via unified API + */ +trait ReadConverterContext { + /** Key pointing to underlying H2OFrame */ + val keyName: String + + /** Chunk Idx/Partition index */ + val chunkIdx: Int + + /** Current row index */ + var rowIdx: Int = 0 + + /** Dummy mutable and reusable holder for String values */ + val valStr = new BufferedString() + + def getByte(columnNum: Int): Byte = getLong(columnNum).toByte + def getShort(columnNum: Int): Short = getLong(columnNum).toShort + def getInt(columnNum: Int): Int = getLong(columnNum).toInt + + def getFloat(columnNum: Int): Float = getDouble(columnNum).toFloat + def getBoolean(columnNum: Int): Boolean = getLong(columnNum) == 1 + def getTimestamp(columnNum: Int): Long = getLong(columnNum) * 1000L + def getUTF8String(columnNum: Int): UTF8String = { + val str = getString(columnNum) + if(str == null){ + null + }else{ + UTF8String.fromString(str) + } + } + + def isNA(columnNum: Int): Boolean + def getLong(columnNum: Int): Long + def getDouble(columnNum: Int): Double + def getString(columnNum: Int): String + + def numRows: Int + def increaseRowIdx() = rowIdx += 1 + def hasNext = rowIdx < numRows +} diff --git a/core/src/main/scala/org/apache/spark/h2o/converters/SparkDataFrameConverter.scala b/core/src/main/scala/org/apache/spark/h2o/converters/SparkDataFrameConverter.scala new file mode 100644 index 000000000..7d76e56f0 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/h2o/converters/SparkDataFrameConverter.scala @@ -0,0 +1,161 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.h2o.converters + +import org.apache.spark._ +import org.apache.spark.h2o.H2OContext +import org.apache.spark.h2o.utils.{H2OSchemaUtils, NodeDesc} +import org.apache.spark.sql.types._ +import org.apache.spark.sql.{DataFrame, H2OFrameRelation, Row, SQLContext} +import water.fvec.{Frame, H2OFrame} + +import scala.collection.immutable + +trait SparkDataFrameConverter extends Logging with ConverterUtils { + + /** + * Create a Spark DataFrame from given H2O frame. + * + * @param hc an instance of H2O context + * @param fr an instance of H2O frame + * @param copyMetadata copy H2O metadata into Spark DataFrame + * @param sqlContext running sqlContext + * @tparam T type of H2O frame + * @return a new DataFrame definition using given H2OFrame as data source + */ + + def toDataFrame[T <: Frame](hc: H2OContext, fr: T, copyMetadata: Boolean)(implicit sqlContext: SQLContext): DataFrame = { + // Relation referencing H2OFrame + val relation = new H2OFrameRelation(fr, copyMetadata)(sqlContext) + sqlContext.baseRelationToDataFrame(relation) + } + + /** Transform Spark's DataFrame into H2O Frame */ + def toH2OFrame(hc: H2OContext, dataFrame: DataFrame, frameKeyName: Option[String]): H2OFrame = { + import H2OSchemaUtils._ + // Cache DataFrame RDD's + val dfRdd = dataFrame.rdd + val keyName = frameKeyName.getOrElse("frame_rdd_" + dfRdd.id) + + // Flattens and expands RDD's schema + val flatRddSchema = expandedSchema(hc.sparkContext, dataFrame) + // Patch the flat schema based on information about types + val fnames = flatRddSchema.map(t => t._2.name).toArray + // Transform datatype into h2o types + val vecTypes = flatRddSchema.indices + .map(idx => { + val f = flatRddSchema(idx) + dataTypeToVecType(f._2.dataType) + }).toArray + + convert[Row](hc, dfRdd, keyName, fnames, vecTypes, perSQLPartition(flatRddSchema)) + } + + /** + * + * @param keyName key of the frame + * @param vecTypes h2o vec types + * @param types flat RDD schema + * @param uploadPlan plan which assigns each partition h2o node where the data from that partition will be uploaded + * @param context spark task context + * @param it iterator over data in the partition + * @return pair (partition ID, number of rows in this partition) + */ + private[this] + def perSQLPartition(types: Seq[(Seq[Int], StructField, Byte)]) + (keyName: String, vecTypes: Array[Byte], uploadPlan: Option[immutable.Map[Int, NodeDesc]]) + (context: TaskContext, it: Iterator[Row]): (Int, Long) = { + val con = ConverterUtils.getWriteConverterContext(uploadPlan, context.partitionId()) + // Creates array of H2O NewChunks; A place to record all the data in this partition + con.createChunks(keyName, vecTypes, context.partitionId()) + + + it.foreach(row => { + var startOfSeq = -1 + // Fill row in the output frame + types.indices.foreach { idx => // Index of column + val field = types(idx) + val path = field._1 + val dataType = field._2.dataType + // Helpers to distinguish embedded collection types + val isAry = field._3 == H2OSchemaUtils.ARRAY_TYPE + val isVec = field._3 == H2OSchemaUtils.VEC_TYPE + val isNewPath = if (idx > 0) path != types(idx - 1)._1 else true + // Reset counter for sequences + if ((isAry || isVec) && isNewPath) startOfSeq = idx + else if (!isAry && !isVec) startOfSeq = -1 + + var i = 0 + var subRow = row + while (i < path.length - 1 && !subRow.isNullAt(path(i))) { + subRow = subRow.getAs[Row](path(i)); + i += 1 + } + val aidx = path(i) // actual index into row provided by path + if (subRow.isNullAt(aidx)) { + con.putNA(idx) + } else { + val ary = if (isAry) subRow.getAs[Seq[_]](aidx) else null + val aryLen = if (isAry) ary.length else -1 + val aryIdx = idx - startOfSeq // shared index to position in array/vector + val vec = if (isVec) subRow.getAs[mllib.linalg.Vector](aidx) else null + if (isAry && aryIdx >= aryLen) con.putNA(idx) + else if (isVec && aryIdx >= vec.size) con.put(idx, 0.0) // Add zeros for vectors + else dataType match { + case BooleanType => con.put(idx, if (isAry) + if (ary(aryIdx).asInstanceOf[Boolean]) 1 else 0 + else if (subRow.getBoolean(aidx)) 1 else 0) + case BinaryType => + case ByteType => con.put(idx, if (isAry) ary(aryIdx).asInstanceOf[Byte] else subRow.getByte(aidx)) + case ShortType => con.put(idx, if (isAry) ary(aryIdx).asInstanceOf[Short] else subRow.getShort(aidx)) + case IntegerType => con.put(idx, if (isAry) ary(aryIdx).asInstanceOf[Int] else subRow.getInt(aidx)) + case LongType => con.put(idx, if (isAry) ary(aryIdx).asInstanceOf[Long] else subRow.getLong(aidx)) + case FloatType => con.put(idx, if (isAry) ary(aryIdx).asInstanceOf[Float] else subRow.getFloat(aidx)) + case DoubleType => con.put(idx, if (isAry) { + ary(aryIdx).asInstanceOf[Double] + } else { + if (isVec) { + subRow.getAs[mllib.linalg.Vector](aidx)(idx - startOfSeq) + } else { + subRow.getDouble(aidx) + } + }) + case StringType => { + val sv = if (isAry) ary(aryIdx).asInstanceOf[String] else subRow.getString(aidx) + // Always produce string vectors + con.put(idx, sv) + } + case TimestampType => con.put(idx, subRow.getAs[java.sql.Timestamp](aidx)) + case _ => con.putNA(idx) + } + } + + } + con.increaseRowCounter() + }) + + //Compress & write data in partitions to H2O Chunks + con.closeChunks() + + // Return Partition number and number of rows in this partition + (context.partitionId, con.numOfRows) + } + +} + +object SparkDataFrameConverter extends SparkDataFrameConverter diff --git a/core/src/main/scala/org/apache/spark/h2o/converters/SupportedRDDConverter.scala b/core/src/main/scala/org/apache/spark/h2o/converters/SupportedRDDConverter.scala new file mode 100644 index 000000000..b66e45a3d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/h2o/converters/SupportedRDDConverter.scala @@ -0,0 +1,129 @@ +/* +* 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.h2o.converters + +import org.apache.spark.h2o._ +import org.apache.spark.mllib.regression.LabeledPoint + +import scala.language.implicitConversions +import scala.reflect.ClassTag +import scala.reflect.runtime.universe._ + + +/** + * This converter just wraps the existing RDD converters and hides the internal RDD covnerters + */ + +trait SupportedRDDConverter{ + /** Transform supported type for conversion to H2OFrame*/ + def toH2OFrame(hc: H2OContext, rdd: SupportedRDD, frameKeyName: Option[String]): H2OFrame = rdd.toH2OFrame(hc, frameKeyName) + + /** Transform H2OFrame to RDD */ + def toRDD[A <: Product: TypeTag: ClassTag, T <: Frame](hc: H2OContext, fr: T): RDD[A] = ProductRDDConverter.toRDD[A, T](hc, fr) +} + +object SupportedRDDConverter extends SupportedRDDConverter + +/** + * Magnet pattern (Type Class pattern) for conversion from various primitive types to their appropriate H2OFrame using + * the method with the same name + */ +trait SupportedRDD { + def toH2OFrame(hc: H2OContext, frameKeyName: Option[String]): H2OFrame +} + +private[this] object SupportedRDD { + + implicit def toH2OFrameFromRDDJavaInt(rdd: RDD[java.lang.Integer]): SupportedRDD = new SupportedRDD { + override def toH2OFrame(hc: H2OContext, frameKeyName: Option[String]): H2OFrame = PrimitiveRDDConverter.toH2OFrame(hc, rdd, frameKeyName) + } + + implicit def toH2OFrameFromRDDJavaByte(rdd: RDD[java.lang.Byte]): SupportedRDD = new SupportedRDD { + override def toH2OFrame(hc: H2OContext, frameKeyName: Option[String]): H2OFrame = PrimitiveRDDConverter.toH2OFrame(hc, rdd, frameKeyName) + } + + implicit def toH2OFrameFromRDDJavaShort(rdd: RDD[java.lang.Short]): SupportedRDD = new SupportedRDD { + override def toH2OFrame(hc: H2OContext, frameKeyName: Option[String]): H2OFrame = PrimitiveRDDConverter.toH2OFrame(hc, rdd, frameKeyName) + } + + implicit def toH2OFrameFromRDDJavaFloat(rdd: RDD[java.lang.Float]): SupportedRDD = new SupportedRDD { + override def toH2OFrame(hc: H2OContext, frameKeyName: Option[String]): H2OFrame = PrimitiveRDDConverter.toH2OFrame(hc, rdd, frameKeyName) + } + + implicit def toH2OFrameFromRDDJavaDouble(rdd: RDD[java.lang.Double]): SupportedRDD = new SupportedRDD { + override def toH2OFrame(hc: H2OContext, frameKeyName: Option[String]): H2OFrame = PrimitiveRDDConverter.toH2OFrame(hc, rdd, frameKeyName) + } + + implicit def toH2OFrameFromRDDJavaLong(rdd: RDD[java.lang.Long]): SupportedRDD = new SupportedRDD { + override def toH2OFrame(hc: H2OContext, frameKeyName: Option[String]): H2OFrame = PrimitiveRDDConverter.toH2OFrame(hc, rdd, frameKeyName) + } + + implicit def toH2OFrameFromRDDJavaBool(rdd: RDD[java.lang.Boolean]): SupportedRDD = new SupportedRDD { + override def toH2OFrame(hc: H2OContext, frameKeyName: Option[String]): H2OFrame = PrimitiveRDDConverter.toH2OFrame(hc, rdd, frameKeyName) + } + + + + + implicit def toH2OFrameFromRDDString(rdd: RDD[String]): SupportedRDD = new SupportedRDD { + override def toH2OFrame(hc: H2OContext, frameKeyName: Option[String]): H2OFrame = PrimitiveRDDConverter.toH2OFrame(hc, rdd, frameKeyName) + } + + implicit def toH2OFrameFromRDDInt(rdd: RDD[Int]): SupportedRDD = new SupportedRDD { + override def toH2OFrame(hc: H2OContext, frameKeyName: Option[String]): H2OFrame = PrimitiveRDDConverter.toH2OFrame(hc, rdd, frameKeyName) + } + + implicit def toH2OFrameFromRDDByte(rdd: RDD[Byte]): SupportedRDD = new SupportedRDD { + override def toH2OFrame(hc: H2OContext, frameKeyName: Option[String]): H2OFrame = PrimitiveRDDConverter.toH2OFrame(hc, rdd, frameKeyName) + } + + implicit def toH2OFrameFromRDDShort(rdd: RDD[Short]): SupportedRDD = new SupportedRDD { + override def toH2OFrame(hc: H2OContext, frameKeyName: Option[String]): H2OFrame = PrimitiveRDDConverter.toH2OFrame(hc, rdd, frameKeyName) + } + + implicit def toH2OFrameFromRDDFloat(rdd: RDD[Float]): SupportedRDD = new SupportedRDD { + override def toH2OFrame(hc: H2OContext, frameKeyName: Option[String]): H2OFrame = PrimitiveRDDConverter.toH2OFrame(hc, rdd, frameKeyName) + } + + implicit def toH2OFrameFromDouble(rdd: RDD[Double]): SupportedRDD = new SupportedRDD { + override def toH2OFrame(hc: H2OContext, frameKeyName: Option[String]): H2OFrame = PrimitiveRDDConverter.toH2OFrame(hc, rdd, frameKeyName) + } + + implicit def toH2OFrameFromRDDLong(rdd: RDD[Long]): SupportedRDD = new SupportedRDD { + override def toH2OFrame(hc: H2OContext, frameKeyName: Option[String]): H2OFrame = PrimitiveRDDConverter.toH2OFrame(hc, rdd, frameKeyName) + } + + implicit def toH2OFrameFromRDDBool(rdd: RDD[Boolean]): SupportedRDD = new SupportedRDD { + override def toH2OFrame(hc: H2OContext, frameKeyName: Option[String]): H2OFrame = PrimitiveRDDConverter.toH2OFrame(hc, rdd, frameKeyName) + } + + implicit def toH2OFrameFromRDDLabeledPoint(rdd: RDD[LabeledPoint]): SupportedRDD = new SupportedRDD { + override def toH2OFrame(hc: H2OContext, frameKeyName: Option[String]): H2OFrame = LabeledPointConverter.toH2OFrame(hc, rdd, frameKeyName) + } + + implicit def toH2OFrameFromRDDTimeStamp(rdd: RDD[java.sql.Timestamp]): SupportedRDD = new SupportedRDD { + override def toH2OFrame(hc: H2OContext, frameKeyName: Option[String]): H2OFrame = PrimitiveRDDConverter.toH2OFrame(hc, rdd, frameKeyName) + } + + implicit def toH2OFrameFromRDDProductNoTypeTag(rdd : RDD[Product]): SupportedRDD = new SupportedRDD { + override def toH2OFrame(hc: H2OContext, frameKeyName: Option[String]): H2OFrame = ProductRDDConverter.toH2OFrame(hc, rdd, frameKeyName) + } + implicit def toH2OFrameFromRDDProduct[A <: Product : TypeTag](rdd : RDD[A]): SupportedRDD = new SupportedRDD { + override def toH2OFrame(hc: H2OContext, frameKeyName: Option[String]): H2OFrame = ProductRDDConverter.toH2OFrame(hc, rdd, frameKeyName) + } +} diff --git a/core/src/main/scala/org/apache/spark/h2o/converters/WriteConverterContext.scala b/core/src/main/scala/org/apache/spark/h2o/converters/WriteConverterContext.scala new file mode 100644 index 000000000..248840657 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/h2o/converters/WriteConverterContext.scala @@ -0,0 +1,38 @@ +/* +* 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.h2o.converters + +/** + * Methods which each WriteConverterContext has to implement. + * + * Write Converter Context is a class which holds the state of connection/chunks and allows us to write/upload to those chunks + * via unified API + */ +trait WriteConverterContext { + def createChunks(keyName: String, vecTypes: Array[Byte], chunkId: Int) + def closeChunks() + def put(columnNum: Int, n: Number) + + def put(columnNum: Int, n: Boolean) + def put(columnNum: Int, n: java.sql.Timestamp) + def put(columnNum: Int, n: String) + def putNA(columnNum: Int) + + def numOfRows: Long + def increaseRowCounter() +} diff --git a/core/src/main/scala/org/apache/spark/h2o/package.scala b/core/src/main/scala/org/apache/spark/h2o/package.scala index c223d3ff4..cb0f98856 100644 --- a/core/src/main/scala/org/apache/spark/h2o/package.scala +++ b/core/src/main/scala/org/apache/spark/h2o/package.scala @@ -25,10 +25,6 @@ package object h2o { //type Key = water.Key type H2O = water.H2O - /* Cannot be enabled since clashes with Spark DataFrame - @deprecated("1.3.0", "Use H2OFrame") - type DataFrame = water.fvec.H2OFrame - */ // Alias for H2OFrame type H2OFrame = water.fvec.H2OFrame // Alias for diff --git a/core/src/main/scala/org/apache/spark/h2o/utils/H2OContextUtils.scala b/core/src/main/scala/org/apache/spark/h2o/utils/H2OContextUtils.scala new file mode 100644 index 000000000..64be1a5da --- /dev/null +++ b/core/src/main/scala/org/apache/spark/h2o/utils/H2OContextUtils.scala @@ -0,0 +1,75 @@ +/* +* 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.h2o.utils + +import org.apache.spark.{Logging, SparkContext} + +/** + * Support methods for H2OContext. + */ +private[spark] trait H2OContextUtils extends Logging{ + + /** + * Open browser for given address. + * + * @param uri address to open in browser, e.g., http://example.com + */ + def openURI(sc: SparkContext, uri: String): Unit = { + import java.awt.Desktop + if (!isTesting(sc)) { + if (Desktop.isDesktopSupported) { + Desktop.getDesktop.browse(new java.net.URI(uri)) + } else { + logWarning(s"Desktop support is missing! Cannot open browser for $uri") + } + } + } + + /** + * Return true if running inside spark/sparkling water test. + * + * @param sc Spark Context + * @return true if the actual run is test run + */ + def isTesting(sc: SparkContext) = sc.conf.contains("spark.testing") || sys.props.contains("spark.testing") + + /** Checks whether version of provided Spark is the same as Spark's version designated for this Sparkling Water version. + * We check for correct version in shell scripts and during the build but we need to do the check also in the code in cases when the user + * executes for example spark-shell command with sparkling water assembly jar passed as --jars and initiates H2OContext. + * (Because in that case no check for correct Spark version has been done so far.) + */ + def isRunningOnCorrectSpark(sc: SparkContext) = sc.version.startsWith(buildSparkMajorVersion) + + + /** + * Returns Major Spark version for which is this version of Sparkling Water designated. + * + * For example, for 1.6.1 returns 1.6 + */ + def buildSparkMajorVersion = { + val stream = getClass.getResourceAsStream("/spark.version") + val version = scala.io.Source.fromInputStream(stream).mkString + if (version.count(_ == '.') == 1) { + // e.g., 1.6 + version + } else { + // 1.4 + version.substring(0, version.lastIndexOf('.')) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/h2o/H2OSchemaUtils.scala b/core/src/main/scala/org/apache/spark/h2o/utils/H2OSchemaUtils.scala similarity index 99% rename from core/src/main/scala/org/apache/spark/h2o/H2OSchemaUtils.scala rename to core/src/main/scala/org/apache/spark/h2o/utils/H2OSchemaUtils.scala index 238840fba..10ae94273 100644 --- a/core/src/main/scala/org/apache/spark/h2o/H2OSchemaUtils.scala +++ b/core/src/main/scala/org/apache/spark/h2o/utils/H2OSchemaUtils.scala @@ -15,15 +15,13 @@ * limitations under the License. */ -package org.apache.spark.h2o +package org.apache.spark.h2o.utils +import org.apache.spark.h2o._ import org.apache.spark.sql._ import org.apache.spark.sql.types._ import org.apache.spark.{SparkContext, mllib} import water.fvec.Vec -import water.parser.Categorical - -import scala.collection.mutable /** * Utilities for working with Spark SQL component. @@ -225,6 +223,7 @@ object H2OSchemaUtils { } /** Collect max size of stored arrays and MLLib vectors. + * * @return list of max sizes for array types, followed by max sizes for vector types. */ private[h2o] def collectMaxArrays(sc: SparkContext, diff --git a/core/src/main/scala/org/apache/spark/h2o/H2OTypeUtils.scala b/core/src/main/scala/org/apache/spark/h2o/utils/H2OTypeUtils.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/h2o/H2OTypeUtils.scala rename to core/src/main/scala/org/apache/spark/h2o/utils/H2OTypeUtils.scala index c0b988c21..7f4100a51 100644 --- a/core/src/main/scala/org/apache/spark/h2o/H2OTypeUtils.scala +++ b/core/src/main/scala/org/apache/spark/h2o/utils/H2OTypeUtils.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.h2o +package org.apache.spark.h2o.utils import water.fvec.Vec diff --git a/core/src/main/scala/org/apache/spark/h2o/utils/NodeDesc.scala b/core/src/main/scala/org/apache/spark/h2o/utils/NodeDesc.scala new file mode 100644 index 000000000..77aac67a8 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/h2o/utils/NodeDesc.scala @@ -0,0 +1,38 @@ +/* +* 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.h2o.utils + +import water.H2ONode + +/** Helper class containing node ID, hostname and port. + * + * @param nodeId In case of external cluster mode the node ID is ID of H2O Node, in the internal cluster mode the ID + * is ID of Spark Executor where corresponding instance is located + * @param hostname hostname of the node + * @param port port of the node + */ +case class NodeDesc(nodeId: String, hostname: String, port: Int) { + override def productPrefix = "" +} + +object NodeDesc { + def fromH2ONode(node: H2ONode): NodeDesc = { + val ipPort = node.getIpPortString.split(":") + NodeDesc(node.index().toString, ipPort(0), Integer.parseInt(ipPort(1))) + } +} diff --git a/core/src/main/scala/org/apache/spark/h2o/ReflectionUtils.scala b/core/src/main/scala/org/apache/spark/h2o/utils/ReflectionUtils.scala similarity index 99% rename from core/src/main/scala/org/apache/spark/h2o/ReflectionUtils.scala rename to core/src/main/scala/org/apache/spark/h2o/utils/ReflectionUtils.scala index 9167ec0cf..69b6be377 100644 --- a/core/src/main/scala/org/apache/spark/h2o/ReflectionUtils.scala +++ b/core/src/main/scala/org/apache/spark/h2o/utils/ReflectionUtils.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.h2o +package org.apache.spark.h2o.utils import water.api.API diff --git a/core/src/main/scala/org/apache/spark/rdd/H2OSchemaRDD.scala b/core/src/main/scala/org/apache/spark/rdd/H2OSchemaRDD.scala deleted file mode 100644 index 7d293f595..000000000 --- a/core/src/main/scala/org/apache/spark/rdd/H2OSchemaRDD.scala +++ /dev/null @@ -1,230 +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.rdd - -import java.util.UUID - -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.h2o.H2OSchemaUtils.vecTypeToDataType -import org.apache.spark.sql.Row -import org.apache.spark.unsafe.types.UTF8String -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, GenericRow} -import org.apache.spark.sql.types._ -import org.apache.spark.{Partition, SparkContext, TaskContext} -import water.fvec.Frame -import water.parser.BufferedString - - -/** - * H2O H2OFrame wrapper providing RDD[Row] API for use in BaseRelation API. - * - * Note: keep this code up-to-date with H2OSchemaRDDInternal - * - * @param frame an instance of H2O frame - * @param requiredColumns list of the columns which should be provided by iterator, null means all - * @param sparkContext a running spark context - */ -private[spark] -class H2OSchemaRDD[T <: Frame](@transient val frame: T, - val requiredColumns: Array[String]) - (@transient sparkContext: SparkContext) - extends RDD[Row](sparkContext, Nil) with H2ORDDLike[T] { - - def this(@transient frame: T) - (@transient sparkContext: SparkContext) = this(frame, null)(sparkContext) - - @DeveloperApi - override def compute(split: Partition, context: TaskContext): Iterator[Row] = { - val kn = frameKeyName - - new H2OChunkIterator[Row] { - override val partIndex: Int = split.index - override val keyName: String = kn - - /** Dummy muttable holder for String values */ - val valStr = new BufferedString() - /* Indexes of selected columns */ - val selectedColumnIndices = if (requiredColumns == null) { - fr.names().indices - } else { - val names = fr.names() - names.indices.filter { idx => - requiredColumns.contains(names(idx)) - } - } - // scalastyle:off - assert(if (requiredColumns == null) - selectedColumnIndices.length == fr.numCols() - else - selectedColumnIndices.length == requiredColumns.length, - "Column selection missing a column!" - ) - // scalastyle:on - /* Types for of columns */ - lazy val types = fr.vecs().map(v => vecTypeToDataType(v)) - - /** Mutable row returned by iterator */ - val mutableRow = new Array[Any](selectedColumnIndices.length) - - override def next(): Row = { - selectedColumnIndices.foreach { i => - val chk = chks(i) - val vec = chk.vec() - val typ = types(i) - if (chk.isNA(row)) { - mutableRow(i) = null - } else { - typ match { - case ByteType => - mutableRow(i) = chk.at8(row).toByte - case ShortType => - mutableRow(i) = chk.at8(row).toShort - case IntegerType => - mutableRow(i) = chk.at8(row).toInt - case LongType => - mutableRow(i) = chk.at8(row) - case FloatType => - mutableRow(i) = chk.atd(row).toFloat - case DoubleType => - mutableRow(i) = chk.atd(row) - case BooleanType => - mutableRow(i) = chk.at8(row) == 1 - case StringType => - val utf8 = if (vec.isCategorical) { - val str = vec.domain()(chk.at8(row).toInt) - UTF8String.fromString(str) - } else if (vec.isString) { - chk.atStr(valStr, row) - UTF8String.fromString(valStr.toString) // TODO improve this. - } else if (vec.isUUID) { - val uuid = new UUID(chk.at16h(row), chk.at16l(row)) - UTF8String.fromString(uuid.toString) - } else null - mutableRow(i) = utf8 - case TimestampType => - mutableRow(i) = chk.at8(row) * 1000L - case _ => ??? - } - } - } - row += 1 - // Return result - new GenericRow(mutableRow) - } - } - } -} - -/** - * H2O H2OFrame wrapper providing RDD[InternalRow] API. - * - * @param frame an H2O frame - * @param requiredColumns list of the columns which should be provided by iterator, null means all - * @param sparkContext a running Spark context - */ -private[spark] -class H2OSchemaRDDInternal[T <: water.fvec.Frame](@transient val frame: T, - val requiredColumns: Array[String]) - (@transient sparkContext: SparkContext) - extends RDD[InternalRow](sparkContext, Nil) with H2ORDDLike[T] { - - def this(@transient frame: T) - (@transient sparkContext: SparkContext) = this(frame, null)(sparkContext) - - @DeveloperApi - override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { - val kn = frameKeyName - - new H2OChunkIterator[InternalRow] { - override val partIndex: Int = split.index - override val keyName: String = kn - - /** Dummy muttable holder for String values */ - val valStr = new BufferedString() - /* Indexes of selected columns */ - val selectedColumnIndices = if (requiredColumns == null) { - fr.names().indices - } else { - requiredColumns.toSeq.map { name => - fr.find(name) - } - } - // Make sure that column selection is consistent - // scalastyle:off - assert(if (requiredColumns == null) - selectedColumnIndices.length == fr.numCols() - else - selectedColumnIndices.length == requiredColumns.length, - "Column selection missing a column!" - ) - // scalastyle:on - /** Types for of columns */ - lazy val types = fr.vecs().map( v => vecTypeToDataType(v)) - /** Mutable row returned by iterator */ - val mutableRow = new GenericMutableRow(selectedColumnIndices.length) - - override def next(): InternalRow = { - selectedColumnIndices.indices.foreach { idx => - val i = selectedColumnIndices(idx) - val chk = chks(i) - val typ = types(i) - val vec = chk.vec() - if (chk.isNA(row)) { - mutableRow.setNullAt(idx) - } else { - typ match { - case ByteType => - mutableRow.setByte(idx, chk.at8(row).asInstanceOf[Byte]) - case ShortType => - mutableRow.setShort(idx, chk.at8(row).asInstanceOf[Short]) - case IntegerType => - mutableRow.setInt(idx, chk.at8(row).asInstanceOf[Int]) - case LongType => - mutableRow.setLong(idx, chk.at8(row)) - case FloatType => - mutableRow.setFloat(idx, chk.atd(row).asInstanceOf[Float]) - case DoubleType => - mutableRow.setDouble(idx, chk.atd(row)) - case BooleanType => - mutableRow.setBoolean(idx, chk.at8(row) == 1) - case StringType => - val utf8 = if (vec.isCategorical) { - val str = vec.domain()(chk.at8(row).asInstanceOf[Int]) - UTF8String.fromString(str) - } else if (vec.isString) { - chk.atStr(valStr, row) - UTF8String.fromString(valStr.toString) - } else if (vec.isUUID) { - val uuid = new UUID(chk.at16h(row), chk.at16l(row)) - UTF8String.fromString(uuid.toString) - } else null - mutableRow.update(idx, utf8) - case TimestampType => - mutableRow.setLong(idx, chk.at8(row) * 1000L) - case _ => ??? - } - } - } - row += 1 - // Return result - mutableRow - } - } - } -} diff --git a/core/src/main/scala/org/apache/spark/sql/H2OSQLContextUtils.scala b/core/src/main/scala/org/apache/spark/sql/H2OSQLContextUtils.scala index 74a575f86..d2285096c 100644 --- a/core/src/main/scala/org/apache/spark/sql/H2OSQLContextUtils.scala +++ b/core/src/main/scala/org/apache/spark/sql/H2OSQLContextUtils.scala @@ -17,8 +17,9 @@ package org.apache.spark.sql -import org.apache.spark.h2o.H2OSchemaUtils -import org.apache.spark.rdd.{H2OSchemaRDD, H2OSchemaRDDInternal, RDD} +import org.apache.spark.h2o.converters.H2ODataFrame +import org.apache.spark.h2o.utils.H2OSchemaUtils +import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.sources.{BaseRelation, PrunedScan, TableScan} import org.apache.spark.sql.types.StructType @@ -49,8 +50,8 @@ case class H2OFrameRelation[T <: Frame](@transient h2oFrame: T, override val schema: StructType = H2OSchemaUtils.createSchema(h2oFrame, copyMetadata) override def buildScan(): RDD[Row] = - new H2OSchemaRDDInternal(h2oFrame)(sqlContext.sparkContext).asInstanceOf[RDD[Row]] + new H2ODataFrame(h2oFrame)(sqlContext.sparkContext).asInstanceOf[RDD[Row]] override def buildScan(requiredColumns: Array[String]): RDD[Row] = - new H2OSchemaRDDInternal(h2oFrame, requiredColumns)(sqlContext.sparkContext).asInstanceOf[RDD[Row]] + new H2ODataFrame(h2oFrame, requiredColumns)(sqlContext.sparkContext).asInstanceOf[RDD[Row]] } diff --git a/core/src/main/scala/water/api/RDDs/RDDsHandler.scala b/core/src/main/scala/water/api/RDDs/RDDsHandler.scala index 09dcf9ba6..d2eadf1f0 100644 --- a/core/src/main/scala/water/api/RDDs/RDDsHandler.scala +++ b/core/src/main/scala/water/api/RDDs/RDDsHandler.scala @@ -17,7 +17,7 @@ package water.api.RDDs import org.apache.spark.SparkContext -import org.apache.spark.h2o.{H2OFrame, H2OContext} +import org.apache.spark.h2o.{H2OContext, H2OFrame} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD import water.Iced @@ -55,15 +55,15 @@ class RDDsHandler(val sc: SparkContext, val h2oContext: H2OContext) extends Hand h2oContext.asH2OFrame(sc.parallelize(Seq.empty[Int]),name) } else { rdd.first() match { - case t if t.isInstanceOf[Double] => h2oContext.asH2OFrame(rdd.asInstanceOf[RDD[Double]],name) - case t if t.isInstanceOf[LabeledPoint] => h2oContext.asH2OFrame(rdd.asInstanceOf[RDD[LabeledPoint]],name) - case t if t.isInstanceOf[Boolean] => h2oContext.asH2OFrame(rdd.asInstanceOf[RDD[Boolean]],name) - case t if t.isInstanceOf[String] => h2oContext.asH2OFrame(rdd.asInstanceOf[RDD[String]],name) - case t if t.isInstanceOf[Int] => h2oContext.asH2OFrame(rdd.asInstanceOf[RDD[Int]],name) - case t if t.isInstanceOf[Float] => h2oContext.asH2OFrame(rdd.asInstanceOf[RDD[Float]],name) - case t if t.isInstanceOf[Long] => h2oContext.asH2OFrame(rdd.asInstanceOf[RDD[Long]],name) - case t if t.isInstanceOf[java.sql.Timestamp] => h2oContext.asH2OFrame(rdd.asInstanceOf[RDD[java.sql.Timestamp]],name) - case t if t.isInstanceOf[Product] => H2OContext.toH2OFrameFromPureProduct(sc, rdd.asInstanceOf[RDD[Product]], name) + case t if t.isInstanceOf[Double] => h2oContext.asH2OFrame(rdd.asInstanceOf[RDD[Double]], name) + case t if t.isInstanceOf[LabeledPoint] => h2oContext.asH2OFrame(rdd.asInstanceOf[RDD[LabeledPoint]], name) + case t if t.isInstanceOf[Boolean] => h2oContext.asH2OFrame(rdd.asInstanceOf[RDD[Boolean]], name) + case t if t.isInstanceOf[String] => h2oContext.asH2OFrame(rdd.asInstanceOf[RDD[String]], name) + case t if t.isInstanceOf[Int] => h2oContext.asH2OFrame(rdd.asInstanceOf[RDD[Int]], name) + case t if t.isInstanceOf[Float] => h2oContext.asH2OFrame(rdd.asInstanceOf[RDD[Float]], name) + case t if t.isInstanceOf[Long] => h2oContext.asH2OFrame(rdd.asInstanceOf[RDD[Long]], name) + case t if t.isInstanceOf[java.sql.Timestamp] => h2oContext.asH2OFrame(rdd.asInstanceOf[RDD[java.sql.Timestamp]], name) + case t if t.isInstanceOf[Product] => h2oContext.asH2OFrame(rdd.asInstanceOf[RDD[Product]], name) case t => throw new IllegalArgumentException(s"Do not understand type $t") } } diff --git a/core/src/main/scala/water/api/RestAPIManager.scala b/core/src/main/scala/water/api/RestAPIManager.scala new file mode 100644 index 000000000..0a78564bf --- /dev/null +++ b/core/src/main/scala/water/api/RestAPIManager.scala @@ -0,0 +1,110 @@ +/* +* 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 water.api + +import org.apache.spark.SparkContext +import org.apache.spark.h2o.H2OContext +import water.api.DataFrames.DataFramesHandler +import water.api.H2OFrames.H2OFramesHandler +import water.api.RDDs.RDDsHandler +import water.api.scalaInt.ScalaCodeHandler + + +object RestAPIManager { + def registerClientWebAPI(h2oContext: H2OContext): Unit = { + if(h2oContext.getConf.isH2OReplEnabled){ + registerScalaIntEndp(h2oContext.sparkContext, h2oContext) + } + registerDataFramesEndp(h2oContext.sparkContext, h2oContext) + registerH2OFramesEndp(h2oContext.sparkContext, h2oContext) + registerRDDsEndp(h2oContext.sparkContext, h2oContext) + } + + private def registerH2OFramesEndp(sc: SparkContext, h2oContext: H2OContext) = { + + val h2oFramesHandler = new H2OFramesHandler(sc, h2oContext) + + def h2oFramesFactory = new HandlerFactory { + override def create(handler: Class[_ <: Handler]): Handler = h2oFramesHandler + } + + RequestServer.registerEndpoint("getDataFrame", "POST", "/3/h2oframes/{h2oframe_id}/dataframe", + classOf[H2OFramesHandler], "toDataFrame", "Transform H2OFrame with given ID to Spark's DataFrame", + h2oFramesFactory) + + } + + private def registerRDDsEndp(sc: SparkContext, h2oContext: H2OContext) = { + + val rddsHandler = new RDDsHandler(sc, h2oContext) + + def rddsFactory = new HandlerFactory { + override def create(aClass: Class[_ <: Handler]): Handler = rddsHandler + } + RequestServer.registerEndpoint("listRDDs", "GET", "/3/RDDs", classOf[RDDsHandler], "list", + "Return all RDDs within Spark cloud", rddsFactory) + + RequestServer.registerEndpoint("getRDD", "POST", "/3/RDDs/{rdd_id}", classOf[RDDsHandler], + "getRDD", "Get RDD with the given ID from Spark cloud", rddsFactory) + + RequestServer.registerEndpoint("rddToH2OFrame", "POST", "/3/RDDs/{rdd_id}/h2oframe", + classOf[RDDsHandler], "toH2OFrame", "Transform RDD with the given ID to H2OFrame", rddsFactory) + + } + + private def registerDataFramesEndp(sc: SparkContext, h2oContext: H2OContext) = { + + val dataFramesHandler = new DataFramesHandler(sc, h2oContext) + + def dataFramesfactory = new HandlerFactory { + override def create(aClass: Class[_ <: Handler]): Handler = dataFramesHandler + } + + RequestServer.registerEndpoint("listDataFrames", "GET", "/3/dataframes", + classOf[DataFramesHandler], "list", "Return all Spark's DataFrames", dataFramesfactory) + + RequestServer.registerEndpoint("getDataFrame", "POST", "/3/dataframes/{dataframe_id}", + classOf[DataFramesHandler], "getDataFrame", "Get Spark's DataFrame with the given ID", dataFramesfactory) + + RequestServer.registerEndpoint("dataFrametoH2OFrame", "POST", + "/3/dataframes/{dataframe_id}/h2oframe", classOf[DataFramesHandler], "toH2OFrame", + "Transform Spark's DataFrame with the given ID to H2OFrame", dataFramesfactory) + + } + + private def registerScalaIntEndp(sc: SparkContext, h2oContext: H2OContext) = { + val scalaCodeHandler = new ScalaCodeHandler(sc, h2oContext) + def scalaCodeFactory = new HandlerFactory { + override def create(aClass: Class[_ <: Handler]): Handler = scalaCodeHandler + } + RequestServer.registerEndpoint("interpretScalaCode", "POST" ,"/3/scalaint/{session_id}", + classOf[ScalaCodeHandler], "interpret", "Interpret the code and return the result", + scalaCodeFactory) + + RequestServer.registerEndpoint("initScalaSession", "POST", "/3/scalaint", + classOf[ScalaCodeHandler], "initSession", "Return session id for communication with scala interpreter", + scalaCodeFactory) + + RequestServer.registerEndpoint("getScalaSessions", "GET" ,"/3/scalaint", + classOf[ScalaCodeHandler], "getSessions", "Return all active session IDs", scalaCodeFactory) + + RequestServer.registerEndpoint("destroyScalaSession", "DELETE", "/3/scalaint/{session_id}", + classOf[ScalaCodeHandler], "destroySession", "Return session id for communication with scala interpreter", + scalaCodeFactory) + } +} diff --git a/core/src/main/scala/water/api/scalaInt/ScalaCodeHandler.scala b/core/src/main/scala/water/api/scalaInt/ScalaCodeHandler.scala index 29fe4fdef..f5a630dbd 100644 --- a/core/src/main/scala/water/api/scalaInt/ScalaCodeHandler.scala +++ b/core/src/main/scala/water/api/scalaInt/ScalaCodeHandler.scala @@ -16,9 +16,9 @@ */ package water.api.scalaInt -import org.apache.spark.h2o.H2OConf +import org.apache.spark.SparkContext +import org.apache.spark.h2o.H2OContext import org.apache.spark.repl.h2o.H2OInterpreter -import org.apache.spark.{SparkConf, SparkContext} import water.Iced import water.api.Handler import water.exceptions.H2ONotFoundArgumentException @@ -28,9 +28,9 @@ import scala.collection.concurrent.TrieMap /** * Handler for all Scala related endpoints */ -class ScalaCodeHandler(val sc: SparkContext) extends Handler with H2OConf{ +class ScalaCodeHandler(val sc: SparkContext, val h2oContext: H2OContext) extends Handler{ - val intrPoolSize = scalaIntDefaultNum + val intrPoolSize = h2oContext.getConf.scalaIntDefaultNum val freeInterpreters = new java.util.concurrent.ConcurrentLinkedQueue[H2OInterpreter] var mapIntr = new TrieMap[Int, H2OInterpreter] var lastIdUsed = 0 @@ -108,9 +108,6 @@ class ScalaCodeHandler(val sc: SparkContext) extends Handler with H2OConf{ lastIdUsed } } - - /* Required Spark config */ - override def sparkConf: SparkConf = sc.getConf } private[api] class IcedCode(val session_id: Int, val code: String) extends Iced[IcedCode] { diff --git a/core/src/main/scala/water/fvec/FrameUtils.scala b/core/src/main/scala/water/fvec/FrameUtils.scala index 5fdd18875..df71832f5 100644 --- a/core/src/main/scala/water/fvec/FrameUtils.scala +++ b/core/src/main/scala/water/fvec/FrameUtils.scala @@ -14,12 +14,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package water.fvec +import org.apache.spark.h2o.utils.NodeDesc + + /** * This is a simple bridge to access package-private/protected members. */ -object FrameUtils { +trait FrameUtils { /** @see Frame#preparePartialFrame */ def preparePartialFrame(fr: Frame, names: Array[String]): Unit = { @@ -45,4 +49,22 @@ object FrameUtils { val chks = vecs.indices.map(idx => vecs(idx).chunkForChunkIdx(cidx)) chks.toArray } + + /** + * Get the home nodes of chunks. Since all the Vecs in a Frame belong to the same Vec.VectorGroup we can only ask one vec + * for its chunks' home nodes (it is then same for the rest of the vectors) + * @param fr frame on which determine home nodes of chunks + * @return mapping of chunk index to node description + */ + def getChunksLocations(fr: Frame): Array[NodeDesc] = { + val chunkCount = fr.anyVec().nChunks() + val cidxToH2ONode = new Array[NodeDesc](chunkCount) + (0 until chunkCount).foreach { cidx => + // SW-172: FIXME this is expensive since we are creating NodeDesc for each call - they should be interned! + cidxToH2ONode(cidx) = NodeDesc.fromH2ONode(fr.anyVec().chunkKey(cidx).home_node()) + } + cidxToH2ONode + } } + +object FrameUtils extends FrameUtils diff --git a/core/src/test/scala/org/apache/spark/h2o/DataSourceTestSuite.scala b/core/src/test/scala/org/apache/spark/h2o/DataSourceTestSuite.scala index fa1313be0..a8fc9b1f0 100644 --- a/core/src/test/scala/org/apache/spark/h2o/DataSourceTestSuite.scala +++ b/core/src/test/scala/org/apache/spark/h2o/DataSourceTestSuite.scala @@ -18,8 +18,8 @@ package org.apache.spark.h2o import org.apache.spark.SparkContext -import org.apache.spark.h2o.util.SharedSparkTestContext -import org.apache.spark.sql.{SaveMode, SQLContext} +import org.apache.spark.h2o.utils.SharedSparkTestContext +import org.apache.spark.sql.SaveMode import org.junit.runner.RunWith import org.scalatest.FunSuite import org.scalatest.junit.JUnitRunner @@ -33,8 +33,7 @@ class DataSourceTestSuite extends FunSuite with SharedSparkTestContext { override def createSparkContext: SparkContext = new SparkContext("local[*]", "test-data-sources", conf = defaultSparkConf) - - + test("Reading H2OFrame using short variant") { val rdd = sc.parallelize(1 to 1000).map( v => IntHolder(Some(v))) val h2oFrame:H2OFrame = hc.asH2OFrame(rdd) @@ -48,7 +47,7 @@ class DataSourceTestSuite extends FunSuite with SharedSparkTestContext { test("Reading H2OFrame using key option") { val rdd = sc.parallelize(1 to 1000).map( v => IntHolder(Some(v))) val h2oFrame:H2OFrame = hc.asH2OFrame(rdd) - val df = sqlc.read.format("h2o").option("key",h2oFrame.key.toString).load() + val df = sqlc.read.format("h2o").option("key", h2oFrame.key.toString).load() assert (df.columns.length == h2oFrame.numCols(), "Number of columns should match") assert (df.columns.sameElements(h2oFrame.names()),"Column names should match") @@ -69,7 +68,8 @@ class DataSourceTestSuite extends FunSuite with SharedSparkTestContext { val rdd = sc.parallelize(1 to 1000).map( v => IntHolder(Some(v))) val df = sqlc.createDataFrame(rdd) df.write.h2o("new_key") - val h2oFrame = DKV.getGet[H2OFrame]("new_key") + + val h2oFrame = DKV.getGet[Frame]("new_key") assert (df.columns.length == h2oFrame.numCols(), "Number of columns should match") assert (df.columns.sameElements(h2oFrame.names()),"Column names should match") assert (df.count() == h2oFrame.numRows(), "Number of rows should match") @@ -85,7 +85,7 @@ class DataSourceTestSuite extends FunSuite with SharedSparkTestContext { val rddNew = sc.parallelize(1 to 1000).map( v => StringHolder(Some(v.toString))) val dfNew = sqlc.createDataFrame(rddNew) - val h2oFrame = DKV.getGet[H2OFrame]("new_key") + val h2oFrame = DKV.getGet[Frame]("new_key") val thrown = intercept[RuntimeException] { dfNew.write.format("h2o").mode(SaveMode.ErrorIfExists).save("new_key") } @@ -104,7 +104,7 @@ class DataSourceTestSuite extends FunSuite with SharedSparkTestContext { dfNew.write.format("h2o").mode(SaveMode.Overwrite).save("new_key") // load new H2O Frame - val h2oFrame = DKV.getGet[H2OFrame]("new_key") + val h2oFrame = DKV.getGet[Frame]("new_key") assert (dfNew.columns.length == h2oFrame.numCols(), "Number of columns should match") assert (dfNew.columns.sameElements(h2oFrame.names()),"Column names should match") @@ -122,7 +122,7 @@ class DataSourceTestSuite extends FunSuite with SharedSparkTestContext { dfNew.write.format("h2o").mode(SaveMode.Ignore).save("new_key") // load new H2O Frame - val h2oFrame = DKV.getGet[H2OFrame]("new_key") + val h2oFrame = DKV.getGet[Frame]("new_key") assert (df.columns.length == h2oFrame.numCols(), "Number of columns should match") assert (df.columns.sameElements(h2oFrame.names()),"Column names should match") diff --git a/core/src/test/scala/org/apache/spark/h2o/H2OConfTestSuite.scala b/core/src/test/scala/org/apache/spark/h2o/H2OConfTestSuite.scala index 741c97098..fd7c866a4 100644 --- a/core/src/test/scala/org/apache/spark/h2o/H2OConfTestSuite.scala +++ b/core/src/test/scala/org/apache/spark/h2o/H2OConfTestSuite.scala @@ -17,17 +17,17 @@ package org.apache.spark.h2o import org.apache.spark.{SparkContext, SparkConf} -import org.apache.spark.h2o.util.SparkTestContext +import org.apache.spark.h2o.utils.SparkTestContext import org.junit.runner.RunWith import org.scalatest.{BeforeAndAfter, Matchers, FunSuite} import org.scalatest.junit.JUnitRunner /** - * Test passing parameters via SparkConf. - */ + * Test passing parameters via SparkConf. + */ @RunWith(classOf[JUnitRunner]) class H2OConfTestSuite extends FunSuite -with Matchers with BeforeAndAfter with SparkTestContext { + with Matchers with BeforeAndAfter with SparkTestContext { test("test H2OConf parameters") { val sparkConf = new SparkConf() @@ -50,28 +50,33 @@ with Matchers with BeforeAndAfter with SparkTestContext { .set("spark.ext.h2o.dummy.rdd.mul.factor", "2") sc = new SparkContext("local", "test-local", sparkConf) - hc = new H2OContext(sc) + + // We don't need to have H2OContext here started and since it has private constructor + // and getOrCreate methods automatically start H2OContext, we use a little bit of reflection + val ctor = classOf[H2OContext].getDeclaredConstructor(classOf[SparkContext], classOf[H2OConf]) + ctor.setAccessible(true) + hc = ctor.newInstance(sc, new H2OConf(sc)) + val conf = hc.getConf // Test passed values - assert(hc.useFlatFile == false) - assert(hc.numH2OWorkers == Some(42)) - assert(hc.clientBasePort == 1267) - assert(hc.nodeBasePort == 32333) - assert(hc.clientIp == Some("10.0.0.100")) - assert(hc.cloudTimeout == 10*1000) - assert(hc.numRddRetries == 2) - assert(hc.cloudName == "test-sparkling-cloud-") - assert(hc.h2oNodeLogLevel == "DEBUG") - assert(hc.h2oClientLogLevel == "DEBUG") - assert(hc.clientNetworkMask == Some("127.0.0.1/32")) - assert(hc.nodeNetworkMask == Some("0.0.0.1/24")) - assert(hc.nthreads == 7) - assert(hc.disableGA == true) - assert(hc.clientWebPort == 13321) - assert(hc.drddMulFactor == 2) + assert(conf.useFlatFile == false) + assert(conf.numH2OWorkers == Some(42)) + assert(conf.clientBasePort == 1267) + assert(conf.nodeBasePort == 32333) + assert(conf.clientIp == Some("10.0.0.100")) + assert(conf.cloudTimeout == 10*1000) + assert(conf.numRddRetries == 2) + assert(conf.cloudName.isDefined) + assert(conf.cloudName == Some("test-sparkling-cloud-")) + assert(conf.h2oNodeLogLevel == "DEBUG") + assert(conf.h2oClientLogLevel == "DEBUG") + assert(conf.clientNetworkMask == Some("127.0.0.1/32")) + assert(conf.nodeNetworkMask == Some("0.0.0.1/24")) + assert(conf.nthreads == 7) + assert(conf.disableGA == true) + assert(conf.clientWebPort == 13321) + assert(conf.drddMulFactor == 2) resetContext() } - - class DummyH2OContext(val sparkConf: SparkConf) extends H2OConf } diff --git a/core/src/test/scala/org/apache/spark/h2o/H2OSchemaRDDTest.scala b/core/src/test/scala/org/apache/spark/h2o/H2OFrameToDataFrameTestSuite.scala similarity index 99% rename from core/src/test/scala/org/apache/spark/h2o/H2OSchemaRDDTest.scala rename to core/src/test/scala/org/apache/spark/h2o/H2OFrameToDataFrameTestSuite.scala index 08156c5ca..fb5aa3cd4 100644 --- a/core/src/test/scala/org/apache/spark/h2o/H2OSchemaRDDTest.scala +++ b/core/src/test/scala/org/apache/spark/h2o/H2OFrameToDataFrameTestSuite.scala @@ -21,8 +21,8 @@ import java.sql.Timestamp import java.util.UUID import hex.splitframe.ShuffleSplitFrame -import org.apache.spark.h2o.H2OSchemaUtils.flatSchema -import org.apache.spark.h2o.util.SharedSparkTestContext +import org.apache.spark.h2o.utils.{SharedSparkTestContext, H2OSchemaUtils} +import H2OSchemaUtils.flatSchema import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.sql.types._ import org.apache.spark.sql.{DataFrame, Row} @@ -41,7 +41,7 @@ import scala.reflect.ClassTag * Testing schema for h2o schema rdd transformation. */ @RunWith(classOf[JUnitRunner]) -class H2OSchemaRDDTest extends FunSuite with SharedSparkTestContext { +class H2OFrameToDataFrameTestSuite extends FunSuite with SharedSparkTestContext { override def createSparkContext: SparkContext = new SparkContext("local[*]", "test-local", conf = defaultSparkConf) @@ -743,7 +743,7 @@ class H2OSchemaRDDTest extends FunSuite with SharedSparkTestContext { } } -object H2OSchemaRDDTest { +object H2OFrameToDataFrameTestSuite { } diff --git a/core/src/test/scala/org/apache/spark/h2o/H2ORDDTest.scala b/core/src/test/scala/org/apache/spark/h2o/H2OFrameToRDDTestSuite.scala similarity index 97% rename from core/src/test/scala/org/apache/spark/h2o/H2ORDDTest.scala rename to core/src/test/scala/org/apache/spark/h2o/H2OFrameToRDDTestSuite.scala index b2eedfa59..cbfd52650 100644 --- a/core/src/test/scala/org/apache/spark/h2o/H2ORDDTest.scala +++ b/core/src/test/scala/org/apache/spark/h2o/H2OFrameToRDDTestSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.h2o import java.sql.Timestamp import org.apache.spark.SparkContext -import org.apache.spark.h2o.util.SharedSparkTestContext +import org.apache.spark.h2o.utils.SharedSparkTestContext import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint import org.junit.runner.RunWith @@ -29,9 +29,8 @@ import water.fvec.Vec import water.parser.{BufferedString, Categorical} /** - * Testing schema for h2o schema rdd transformation. + * Testing schema for rdd to h2o frame transformations. */ -// FIXME this should be only trait but used in different SparkContext @RunWith(classOf[JUnitRunner]) class H2ORDDTest extends FunSuite with SharedSparkTestContext { @@ -112,9 +111,8 @@ class H2ORDDTest extends FunSuite with SharedSparkTestContext { test("PUBDEV-458 - from Rdd[IntHolder] to H2OFrame and back") { val h2oContext = hc import h2oContext.implicits._ - val rdd = sc.parallelize(1 to 100, 10).map(i => IntHolder(Some(i))) + val rdd = sc.parallelize(1 to 1000000, 10).map(i => IntHolder(Some(i))) val h2oFrame:H2OFrame = rdd - val back2rdd = hc.asRDD[PUBDEV458Type](h2oFrame) assert(rdd.count == h2oFrame.numRows(), "Number of rows should match") assert(back2rdd.count == h2oFrame.numRows(), "Number of rows should match") diff --git a/core/src/test/scala/org/apache/spark/h2o/H2OSchemaUtilsTestSuite.scala b/core/src/test/scala/org/apache/spark/h2o/H2OSchemaUtilsTestSuite.scala index b700904fe..ad995e93e 100644 --- a/core/src/test/scala/org/apache/spark/h2o/H2OSchemaUtilsTestSuite.scala +++ b/core/src/test/scala/org/apache/spark/h2o/H2OSchemaUtilsTestSuite.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.h2o +import org.apache.spark.h2o.utils.H2OSchemaUtils import org.apache.spark.sql.types.{DoubleType, IntegerType, StringType, StructField, StructType} import org.junit.runner.RunWith import org.scalatest.FunSuite diff --git a/core/src/test/scala/org/apache/spark/h2o/util/PerTestSparkTestContext.scala b/core/src/test/scala/org/apache/spark/h2o/utils/PerTestSparkTestContext.scala similarity index 81% rename from core/src/test/scala/org/apache/spark/h2o/util/PerTestSparkTestContext.scala rename to core/src/test/scala/org/apache/spark/h2o/utils/PerTestSparkTestContext.scala index aaa46d2b2..9ec3b020f 100644 --- a/core/src/test/scala/org/apache/spark/h2o/util/PerTestSparkTestContext.scala +++ b/core/src/test/scala/org/apache/spark/h2o/utils/PerTestSparkTestContext.scala @@ -14,10 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.h2o.util +package org.apache.spark.h2o.utils import org.apache.spark.SparkContext -import org.apache.spark.h2o.H2OContext +import org.apache.spark.h2o.{H2OConf, H2OContext} +import org.apache.spark.sql.SQLContext import org.scalatest.Suite /** This fixture create a Spark context once and share it over whole run of test suite. @@ -26,12 +27,13 @@ import org.scalatest.Suite trait PerTestSparkTestContext extends SparkTestContext { self: Suite => def createSparkContext:SparkContext - def createH2OContext(sc:SparkContext):H2OContext = H2OContext.getOrCreate(sc) + def createH2OContext(sc: SparkContext, conf: H2OConf):H2OContext = H2OContext.getOrCreate(sc) override protected def beforeEach(): Unit = { super.beforeEach() sc = createSparkContext - hc = createH2OContext(sc) + sqlc = SQLContext.getOrCreate(sc) + hc = createH2OContext(sc, new H2OConf(sc)) } override protected def afterEach(): Unit = { diff --git a/core/src/test/scala/org/apache/spark/h2o/util/SharedSparkTestContext.scala b/core/src/test/scala/org/apache/spark/h2o/utils/SharedSparkTestContext.scala similarity index 73% rename from core/src/test/scala/org/apache/spark/h2o/util/SharedSparkTestContext.scala rename to core/src/test/scala/org/apache/spark/h2o/utils/SharedSparkTestContext.scala index e03ffbc78..493f297e9 100644 --- a/core/src/test/scala/org/apache/spark/h2o/util/SharedSparkTestContext.scala +++ b/core/src/test/scala/org/apache/spark/h2o/utils/SharedSparkTestContext.scala @@ -14,27 +14,34 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.h2o.util +package org.apache.spark.h2o.utils import org.apache.spark.SparkContext -import org.apache.spark.h2o.H2OContext +import org.apache.spark.h2o.backends.SharedH2OConf._ +import org.apache.spark.h2o.{H2OConf, H2OContext} import org.apache.spark.sql.SQLContext import org.scalatest.Suite - -/** This fixture create a Spark context once and share it over whole run of test suite. */ +/** + * Helper trait to simplify initialization and termination of Spark/H2O contexts. + * + */ trait SharedSparkTestContext extends SparkTestContext { self: Suite => + def createSparkContext:SparkContext - def createH2OContext(sc:SparkContext):H2OContext = H2OContext.getOrCreate(sc) + + def createH2OContext(sc: SparkContext, conf: H2OConf): H2OContext = { + H2OContext.getOrCreate(sc, conf) + } override def beforeAll(): Unit = { super.beforeAll() sc = createSparkContext sqlc = SQLContext.getOrCreate(sc) - hc = createH2OContext(sc) + hc = createH2OContext(sc, new H2OConf(sc)) } - override protected def afterAll(): Unit = { + override def afterAll(): Unit = { resetContext() super.afterAll() } diff --git a/core/src/test/scala/org/apache/spark/h2o/util/SparkTestContext.scala b/core/src/test/scala/org/apache/spark/h2o/utils/SparkTestContext.scala similarity index 85% rename from core/src/test/scala/org/apache/spark/h2o/util/SparkTestContext.scala rename to core/src/test/scala/org/apache/spark/h2o/utils/SparkTestContext.scala index 8949bf575..8799391fa 100644 --- a/core/src/test/scala/org/apache/spark/h2o/util/SparkTestContext.scala +++ b/core/src/test/scala/org/apache/spark/h2o/utils/SparkTestContext.scala @@ -14,19 +14,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.h2o.util -import io.netty.util.internal.logging.{Slf4JLoggerFactory, InternalLoggerFactory} +package org.apache.spark.h2o.utils + +import java.net.InetAddress + +import io.netty.util.internal.logging.{InternalLoggerFactory, Slf4JLoggerFactory} +import org.apache.spark.h2o.H2OContext import org.apache.spark.sql.SQLContext import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.h2o.H2OContext -import org.scalatest.{Suite, BeforeAndAfterAll, BeforeAndAfterEach} +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, Suite} /** * Helper trait to simplify initialization and termination of Spark/H2O contexts. * */ trait SparkTestContext extends BeforeAndAfterEach with BeforeAndAfterAll { self: Suite => + @transient var sc: SparkContext = _ @transient var hc: H2OContext = _ @transient implicit var sqlc: SQLContext = _ @@ -42,6 +46,7 @@ trait SparkTestContext extends BeforeAndAfterEach with BeforeAndAfterAll { self: sc = null hc = null } + def defaultSparkConf = new SparkConf() .set("spark.ext.h2o.disable.ga", "true") .set("spark.driver.memory", "2G") @@ -50,6 +55,8 @@ trait SparkTestContext extends BeforeAndAfterEach with BeforeAndAfterAll { self: .set("spark.ext.h2o.client.log.level", "DEBUG") .set("spark.ext.h2o.repl.enabled","false") // disable repl in tests .set("spark.scheduler.minRegisteredResourcesRatio", "1") + .set("spark.ext.h2o.backend.cluster.mode", sys.props.getOrElse("spark.ext.h2o.backend.cluster.mode", "internal")) + .set("spark.ext.h2o.client.ip", InetAddress.getLocalHost.getHostAddress) } object SparkTestContext { @@ -71,4 +78,3 @@ object SparkTestContext { } } - diff --git a/core/src/test/scala/water/api/DataFramesHandlerSuite.scala b/core/src/test/scala/water/api/DataFramesHandlerSuite.scala index 4e0300e94..df10aab24 100644 --- a/core/src/test/scala/water/api/DataFramesHandlerSuite.scala +++ b/core/src/test/scala/water/api/DataFramesHandlerSuite.scala @@ -20,7 +20,7 @@ import java.io.File import com.google.gson.JsonParser import org.apache.spark.SparkContext -import org.apache.spark.h2o.util.SharedSparkTestContext +import org.apache.spark.h2o.utils.SharedSparkTestContext import org.apache.spark.sql.types.{DataType, Metadata, StructField, StructType} import org.junit.runner.RunWith import org.scalatest.FunSuite @@ -46,7 +46,7 @@ class DataFramesHandlerSuite extends FunSuite with SharedSparkTestContext { val df = rdd.toDF("nums") df.registerTempTable(rid) - val dataFramesHandler = new DataFramesHandler(sc,hc) + val dataFramesHandler = new DataFramesHandler(sc, hc) val req = new DataFramesV3 val result = dataFramesHandler.list(3, req) @@ -109,18 +109,13 @@ class DataFramesHandlerSuite extends FunSuite with SharedSparkTestContext { req.h2oframe_id ="requested_name" val result = dataFramesHandler.toH2OFrame(3, req) - // create h2o frame for the given id - val value = DKV.get(result.h2oframe_id) - val h2oFrame: H2OFrame = value.className() match { - case name if name.equals(classOf[Frame].getName) => { - val h2oContext = hc - import h2oContext.implicits._ - value.get[Frame]() - } - case name if name.equals(classOf[H2OFrame].getName) => value.get[H2OFrame]() - } + // get h2o frame for the given id + val h2oContext = hc + import h2oContext.implicits._ + val h2oFrame = DKV.getGet[Frame](result.h2oframe_id) + assert (h2oFrame.key.toString == "requested_name", "H2OFrame ID should be equal to \"requested_name\"") - assert (h2oFrame.numCols()==df.columns.size, "Number of columns should match") + assert (h2oFrame.numCols()==df.columns.length, "Number of columns should match") assert (h2oFrame.names().sameElements(df.columns),"Column names should match") assert (h2oFrame.numRows() == df.count(), "Number of rows should match") } @@ -146,8 +141,6 @@ class DataFramesHandlerSuite extends FunSuite with SharedSparkTestContext { } def parseSchema(schemaString: String) : StructType = { - import com.google.gson.Gson - val gson = new Gson() val parser = new JsonParser val obj = parser.parse(schemaString).getAsJsonObject val fields = obj.get("fields").getAsJsonArray diff --git a/core/src/test/scala/water/api/H2OFramesHandlerSuite.scala b/core/src/test/scala/water/api/H2OFramesHandlerSuite.scala index ea3d39f37..14eb3c078 100644 --- a/core/src/test/scala/water/api/H2OFramesHandlerSuite.scala +++ b/core/src/test/scala/water/api/H2OFramesHandlerSuite.scala @@ -19,9 +19,7 @@ package water.api import java.io.File import org.apache.spark.SparkContext -import org.apache.spark.h2o._ -import org.apache.spark.h2o.util.{SharedSparkTestContext, SparkTestContext} -import org.apache.spark.sql.SQLContext +import org.apache.spark.h2o.utils.SharedSparkTestContext import org.junit.runner.RunWith import org.scalatest.FunSuite import org.scalatest.junit.JUnitRunner @@ -49,10 +47,10 @@ class H2OFramesHandlerSuite extends FunSuite with SharedSparkTestContext { // get the data frame using obtained id val df = sqlc.table(result.dataframe_id) assert (sqlc.tableNames().contains("requested_name"), "DataFrame should be stored in table named \"requested_name\"") - assert (df.columns.size == h2oFrame.numCols(), "Number of columns should match") + assert (df.columns.length == h2oFrame.numCols(), "Number of columns should match") assert (df.columns.sameElements(h2oFrame.names()),"Column names should match") assert (df.count() == h2oFrame.numRows(), "Number of rows should match") - assert (sqlc.tableNames().size == 1, "Number of stored DataFrames should be 1") + assert (sqlc.tableNames().length == 1, "Number of stored DataFrames should be 1") } diff --git a/core/src/test/scala/water/api/RDDsHandlerSuite.scala b/core/src/test/scala/water/api/RDDsHandlerSuite.scala index 8b8b66a6a..b5be57fb7 100644 --- a/core/src/test/scala/water/api/RDDsHandlerSuite.scala +++ b/core/src/test/scala/water/api/RDDsHandlerSuite.scala @@ -16,12 +16,12 @@ */ package water.api -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.h2o.util.SharedSparkTestContext +import org.apache.spark.SparkContext +import org.apache.spark.h2o.utils.SharedSparkTestContext import org.junit.runner.RunWith import org.scalatest.FunSuite import org.scalatest.junit.JUnitRunner -import water.api.RDDs.{RDDV3, RDD2H2OFrameIDV3, RDDsHandler, RDDsV3} +import water.api.RDDs.{RDD2H2OFrameIDV3, RDDV3, RDDsHandler, RDDsV3} import water.exceptions.H2ONotFoundArgumentException /** @@ -30,8 +30,7 @@ import water.exceptions.H2ONotFoundArgumentException @RunWith(classOf[JUnitRunner]) class RDDsHandlerSuite extends FunSuite with SharedSparkTestContext { - val sparkConf = new SparkConf().setMaster("local[*]").setAppName("test-local") - override def createSparkContext: SparkContext = new SparkContext(sparkConf) + override def createSparkContext: SparkContext = new SparkContext("local[*]", "test-local", conf = defaultSparkConf) test("RDDsHandler.list() method") { val rname = "Test" @@ -87,18 +86,18 @@ class RDDsHandlerSuite extends FunSuite with SharedSparkTestContext { val rname = "Test" val rpart = 21 - val rdd = sc.parallelize(Seq(A(1,"A"),A(2,"B"),A(3,"C")),rpart).setName(rname).cache() + val rdd = sc.parallelize(Seq(A(1,"A"),A(2,"B"),A(3,"C")), rpart).setName(rname).cache() val rddsHandler = new RDDsHandler(sc, hc) val rddReq = new RDD2H2OFrameIDV3 rddReq.rdd_id = rdd.id rddReq.h2oframe_id = "requested_name" - val result = rddsHandler.toH2OFrame(3,rddReq) + val result = rddsHandler.toH2OFrame(3, rddReq) val h2oframe = hc.asH2OFrame(result.h2oframe_id) assert (h2oframe.key.toString == "requested_name", "H2OFrame ID should be equal to \"requested_name\"") assert (h2oframe.numCols() == 2, "Number of columns should match") - assert (h2oframe.names().sortWith((x, y) => x < y).sameElements(Seq("f0","f1")),"Column names should match") + assert (h2oframe.names().sorted.sameElements(Seq("f0","f1")),"Column names should match") assert (h2oframe.numRows() == rdd.count(), "Number of rows should match") } diff --git a/core/src/test/scala/water/api/ScalaCodeHandlerSuite.scala b/core/src/test/scala/water/api/ScalaCodeHandlerSuite.scala index de45efa03..0dda6094f 100644 --- a/core/src/test/scala/water/api/ScalaCodeHandlerSuite.scala +++ b/core/src/test/scala/water/api/ScalaCodeHandlerSuite.scala @@ -17,8 +17,8 @@ package water.api import org.apache.spark.SparkContext -import org.apache.spark.h2o.H2OContext -import org.apache.spark.h2o.util.SharedSparkTestContext +import org.apache.spark.h2o.utils.SharedSparkTestContext +import org.apache.spark.h2o.{H2OConf, H2OContext} import org.junit.runner.RunWith import org.scalatest.junit.JUnitRunner import org.scalatest.{BeforeAndAfterEach, FunSuite} @@ -32,10 +32,12 @@ import water.exceptions.H2ONotFoundArgumentException class ScalaCodeHandlerSuite extends FunSuite with SharedSparkTestContext with BeforeAndAfterEach { var scalaCodeHandler: ScalaCodeHandler = _ - override def createSparkContext: SparkContext = new SparkContext("local[*]", "test-local", conf = defaultSparkConf.set("spark.ext.h2o.repl.enabled","true")) + override def createSparkContext: SparkContext = new SparkContext("local[*]", "test-local", conf = defaultSparkConf.set("spark.ext.h2o.repl.enabled", "true")) + + override def createH2OContext(sc: SparkContext, conf: H2OConf): H2OContext = H2OContext.getOrCreate(sc, conf) override protected def beforeEach(): Unit = { - scalaCodeHandler = new ScalaCodeHandler(sc) + scalaCodeHandler = new ScalaCodeHandler(sc, hc) } test("ScalaCodeHandler after initialization"){ @@ -50,27 +52,27 @@ class ScalaCodeHandlerSuite extends FunSuite with SharedSparkTestContext with Be assert(result.session_id == 1,"First id should be equal to 1") // new interpreter is automatically created, so the last ID used should be equal to 2 assert(scalaCodeHandler.mapIntr.size == 1, "Number of currently used interpreters should be equal to 1") - assert(scalaCodeHandler.mapIntr.get(1).nonEmpty, "The value in the interpreters hashmap with the key 1 should not be empty") + assert(scalaCodeHandler.mapIntr.get(1).nonEmpty, "The value in the interpreters hash map with the key 1 should not be empty") assert(scalaCodeHandler.mapIntr.get(1).get.sessionId == 1, "ID attached to the interpreter should be equal to 1") } test("ScalaCodeHandler.destroySession() method, destroy existing session"){ // create new session val reqSession = new ScalaSessionIdV3 - scalaCodeHandler.initSession(3,reqSession) + scalaCodeHandler.initSession(3, reqSession) val reqMsg = new ScalaSessionIdV3 - reqMsg.session_id=reqSession.session_id - scalaCodeHandler.destroySession(3,reqMsg) + reqMsg.session_id = reqSession.session_id + scalaCodeHandler.destroySession(3, reqMsg) assert(scalaCodeHandler.mapIntr.isEmpty, "Number of currently used interpreters should be equal to 0") assert(scalaCodeHandler.mapIntr.get(1).isEmpty, "The value in the interpreters hashmap with the key 1 should be empty") } test("ScalaCodeHandler.destroySession() method, destroy non-existing session"){ val reqMsg = new ScalaSessionIdV3 - reqMsg.session_id=3 + reqMsg.session_id = 3 intercept[H2ONotFoundArgumentException] { - scalaCodeHandler.destroySession(3,reqMsg) + scalaCodeHandler.destroySession(3, reqMsg) } assert(scalaCodeHandler.mapIntr.isEmpty, "Number of currently used interpreters should be equal to 0") assert(scalaCodeHandler.mapIntr.get(3).isEmpty, "The value in the interpreters hashmap with the key 3 should be empty") @@ -79,105 +81,105 @@ class ScalaCodeHandlerSuite extends FunSuite with SharedSparkTestContext with Be test("ScalaCodeHandler.getSessions() method"){ // create first interpreter val reqSession1 = new ScalaSessionIdV3 - scalaCodeHandler.initSession(3,reqSession1) + scalaCodeHandler.initSession(3, reqSession1) // create second interpreter val reqSession2 = new ScalaSessionIdV3 - scalaCodeHandler.initSession(3,reqSession2) + scalaCodeHandler.initSession(3, reqSession2) val req = new ScalaSessionsV3 - val result = scalaCodeHandler.getSessions(3,req) + val result = scalaCodeHandler.getSessions(3, req) - assert(result.sessions.sorted.sameElements(Array(1,2)),"Array of active sessions should contain 1 and 2") + assert(result.sessions.sorted.sameElements(Array(1, 2)), "Array of active sessions should contain 1 and 2") assert(scalaCodeHandler.mapIntr.size == 2, "Number of currently used interpreters should be equal to 2") } test("ScalaCodeHandler.interpret() method, printing"){ // create interpreter val reqSession = new ScalaSessionIdV3 - scalaCodeHandler.initSession(3,reqSession) + scalaCodeHandler.initSession(3, reqSession) val req = new ScalaCodeV3 req.session_id = reqSession.session_id req.code = "println(\"text\")" - val result = scalaCodeHandler.interpret(3,req) + val result = scalaCodeHandler.interpret(3, req) - assert(result.output.equals("text\n"),"Printed output should be equal to \"text\"") - assert(result.status.equals("Success"),"Status should be Success") - assert(result.response.equals(""),"Response should be empty") + assert(result.output.equals("text\n"), "Printed output should be equal to \"text\"") + assert(result.status.equals("Success"), "Status should be Success") + assert(result.response.equals(""), "Response should be empty") } test("ScalaCodeHandler.interpret() method, using unknown function"){ // create interpreter val reqSession = new ScalaSessionIdV3 - scalaCodeHandler.initSession(3,reqSession) + scalaCodeHandler.initSession(3, reqSession) val req = new ScalaCodeV3 req.session_id = reqSession.session_id req.code = "foo" - val result = scalaCodeHandler.interpret(3,req) + val result = scalaCodeHandler.interpret(3, req) - assert(result.output.equals(""),"Printed output should be empty") - assert(result.status.equals("Error"),"Status should be Error") - assert(result.response.equals(":33: error: not found: value foo\n foo\n ^\n"),"Response should not be empty") + assert(result.output.equals(""), "Printed output should be empty") + assert(result.status.equals("Error"), "Status should be Error") + assert(result.response.equals(":33: error: not found: value foo\n foo\n ^\n"), "Response should not be empty") } test("ScalaCodeHandler.interpret() method, using previously defined class"){ // create interpreter val reqSession = new ScalaSessionIdV3 - scalaCodeHandler.initSession(3,reqSession) + scalaCodeHandler.initSession(3, reqSession) val req1 = new ScalaCodeV3 req1.session_id = reqSession.session_id req1.code = "case class Foo(num: Int)" - val result1 = scalaCodeHandler.interpret(3,req1) + val result1 = scalaCodeHandler.interpret(3, req1) - assert(result1.output.equals(""),"Printed output should be empty") - assert(result1.status.equals("Success"),"Status should be Success") - assert(result1.response.equals("defined class Foo\n"),"Response should not be empty") + assert(result1.output.equals(""), "Printed output should be empty") + assert(result1.status.equals("Success"), "Status should be Success") + assert(result1.response.equals("defined class Foo\n"), "Response should not be empty") val req2= new ScalaCodeV3 req2.session_id = reqSession.session_id req2.code = "val num = Foo(42)" - val result2 = scalaCodeHandler.interpret(3,req2) + val result2 = scalaCodeHandler.interpret(3, req2) - assert(result2.output.equals(""),"Printed output should equal to text") - assert(result2.status.equals("Success"),"Status should be Success") - assert(result2.response.equals("num: Foo = Foo(42)\n"),"Response should not be empty") + assert(result2.output.equals(""), "Printed output should equal to text") + assert(result2.status.equals("Success"), "Status should be Success") + assert(result2.response.equals("num: Foo = Foo(42)\n"), "Response should not be empty") } - test("ScalaCodeHandler.interpret() method, using sqlContext,h2oContext and sparkContext"){ + test("ScalaCodeHandler.interpret() method, using sqlContext, h2oContext and sparkContext"){ // create interpreter val reqSession = new ScalaSessionIdV3 - scalaCodeHandler.initSession(3,reqSession) + scalaCodeHandler.initSession(3, reqSession) val req1 = new ScalaCodeV3 req1.session_id = reqSession.session_id req1.code = "val rdd = sc.parallelize(1 to 100, 8).map(v=>v+10);rdd.cache" - val result1 = scalaCodeHandler.interpret(3,req1) - assert(result1.output.equals(""),"Printed output should be empty") - assert(result1.status.equals("Success"),"Status should be Success") + val result1 = scalaCodeHandler.interpret(3, req1) + assert(result1.output.equals(""), "Printed output should be empty") + assert(result1.status.equals("Success"), "Status should be Success ") assert(result1.response.contains("rdd: org.apache.spark.rdd.RDD[Int] = MapPartitionsRDD"), "Response should not be empty") val req2 = new ScalaCodeV3 req2.session_id = reqSession.session_id req2.code = "val h2oFrame = h2oContext.asH2OFrame(rdd)" - val result2 = scalaCodeHandler.interpret(3,req2) - assert(result2.output.equals(""),"Printed output should be empty") - assert(result2.status.equals("Success"),"Status should be Success") - assert(!result2.response.equals(""),"Response should not be empty") + val result2 = scalaCodeHandler.interpret(3, req2) + assert(result2.output.equals(""), "Printed output should be empty") + assert(result2.status.equals("Success"), "Status should be Success ") + assert(!result2.response.equals(""), "Response should not be empty") val req3 = new ScalaCodeV3 req3.session_id = reqSession.session_id // this code is using implicitly sqlContext req3.code = "val dataframe = h2oContext.asDataFrame(h2oFrame)" - val result3 = scalaCodeHandler.interpret(3,req3) - assert(result3.output.equals(""),"Printed output should be empty") - assert(result3.status.equals("Success"),"Status should be Success") - assert(!result3.response.equals(""),"Response should not be empty") + val result3 = scalaCodeHandler.interpret(3, req3) + assert(result3.output.equals(""), "Printed output should be empty") + assert(result3.status.equals("Success"), "Status should be Success 3") + assert(!result3.response.equals(""), "Response should not be empty") } } diff --git a/core/src/test/scala/water/api/SupportAPISuite.scala b/core/src/test/scala/water/api/SupportAPISuite.scala index 2534c3623..8aad0db91 100644 --- a/core/src/test/scala/water/api/SupportAPISuite.scala +++ b/core/src/test/scala/water/api/SupportAPISuite.scala @@ -17,7 +17,7 @@ package water.api import org.apache.spark.SparkContext -import org.apache.spark.h2o.util.SharedSparkTestContext +import org.apache.spark.h2o.utils.SharedSparkTestContext import org.scalatest.FunSuite import water.fvec.{AppendableVec, Frame, NewChunk, Vec} import water.munging.JoinMethod diff --git a/examples/build.gradle b/examples/build.gradle index 7f64f0925..a4ce8f7ee 100644 --- a/examples/build.gradle +++ b/examples/build.gradle @@ -21,11 +21,17 @@ dependencies { // And use scalatest for Scala testing testCompile "org.scalatest:scalatest_${scalaBaseVersion}:2.2.1" testCompile "junit:junit:4.11" - + + // to reuse some utils test classes defined in the core + testCompile project(path: ':sparkling-water-core', configuration: 'testArchives') + // Integration tests requirements integTestCompile "org.scalatest:scalatest_${scalaBaseVersion}:2.2.1" integTestCompile "junit:junit:4.11" + // to reuse some utils test classes defined in the core + integTestCompile project(path: ':sparkling-water-core', configuration: 'testArchives') + // To enable Idea compiler even for integTestCode which use mllib /*integTestCompile*/ compile "org.apache.spark:spark-mllib_${scalaBaseVersion}:${sparkVersion}" @@ -34,7 +40,6 @@ dependencies { } integTest { -// Pass references to libraries to test launcher systemProperty "spark.testing", "true" systemProperty "spark.test.home", "${sparkHome}" systemProperty "sparkling.test.hdp.version", "${hdpVersion}" @@ -66,14 +71,4 @@ scriptsTest { maxHeapSize = "4g" // Working dir will be root project workingDir = rootDir -} - -String detectEnvironment(String defaultEnv = "local") { - String denv = [ project.hasProperty("sparklingTestEnv") ? project["sparklingTestEnv"] : null, - System.properties["sparklingTestEnv"], - defaultEnv - ].find { h -> h!=null } // first match - // Return env - logger.info("* Detected '$denv' Sparkling test environment (configure via property 'sparklingTestEnv')") - denv -} +} \ No newline at end of file diff --git a/ml/examples/pipelines/hamOrSpam.script.scala b/examples/pipelines/hamOrSpam.script.scala similarity index 100% rename from ml/examples/pipelines/hamOrSpam.script.scala rename to examples/pipelines/hamOrSpam.script.scala diff --git a/examples/scripts/StrataAirlines.script.scala b/examples/scripts/StrataAirlines.script.scala index 29a961c4a..1cd378969 100644 --- a/examples/scripts/StrataAirlines.script.scala +++ b/examples/scripts/StrataAirlines.script.scala @@ -13,7 +13,7 @@ import org.apache.spark.SparkFiles import org.apache.spark.h2o._ import org.apache.spark.examples.h2o._ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.{DataFrame, SQLContext} import water.Key import java.io.File @@ -40,8 +40,8 @@ val airlinesData = new H2OFrame(new File(SparkFiles.get("year2005.csv.gz"))) val wrawdata = sc.textFile(SparkFiles.get("Chicago_Ohare_International_Airport.csv"),8).cache() val weatherTable = wrawdata.map(_.split(",")).map(row => WeatherParse(row)).filter(!_.isWrongRow()) -// Transfer data from H2O to Spark RDD -val airlinesTable : RDD[Airlines] = asRDD[Airlines](airlinesData) +// Transfer data from H2O to Spark DataFrame +val airlinesTable = h2oContext.asDataFrame(airlinesData).map(row => AirlinesParse(row)) val flightsToORD = airlinesTable.filter(f => f.Dest==Some("ORD")) // Use Spark SQL to join flight and weather data in spark @@ -61,6 +61,7 @@ val bigTable = sqlContext.sql( |JOIN WeatherORD w |ON f.Year=w.Year AND f.Month=w.Month AND f.DayofMonth=w.Day""".stripMargin) + val trainFrame:H2OFrame = bigTable trainFrame.replace(19, trainFrame.vec("IsDepDelayed").toCategoricalVec) trainFrame.update() diff --git a/examples/src/integTest/scala/water/sparkling/itest/ItegTestHelper.scala b/examples/src/integTest/scala/water/sparkling/itest/ItegTestHelper.scala index 4fc4ca0ba..f68c9de37 100644 --- a/examples/src/integTest/scala/water/sparkling/itest/ItegTestHelper.scala +++ b/examples/src/integTest/scala/water/sparkling/itest/ItegTestHelper.scala @@ -1,5 +1,6 @@ package water.sparkling.itest +import java.io.{PrintWriter, StringWriter} import org.scalatest.{BeforeAndAfterEach, Suite, Tag} import scala.collection.mutable @@ -10,7 +11,7 @@ import scala.util.Random */ trait IntegTestHelper extends BeforeAndAfterEach { self: Suite => - private var testEnv:IntegTestEnv = _ + private var testEnv: IntegTestEnv = _ /** Launch given class name via SparkSubmit and use given environment * to configure SparkSubmit command line. @@ -113,3 +114,19 @@ trait IntegTestHelper extends BeforeAndAfterEach { self: Suite => object YarnTest extends Tag("water.sparkling.itest.YarnTest") object LocalTest extends Tag("water.sparkling.itest.LocalTest") object StandaloneTest extends Tag("water.sparkling.itest.StandaloneTest") + + +trait IntegTestStopper extends org.apache.spark.Logging { + + def exitOnException(f: => Unit): Unit ={ + try { + f + } catch { + case t: Throwable => { + logError("Test throws exception!", t) + t.printStackTrace() + water.H2O.exit(-1) + } + } + } +} \ No newline at end of file diff --git a/examples/src/integTest/scala/water/sparkling/itest/local/AirlinesWithWeatherDemo2Suite.scala b/examples/src/integTest/scala/water/sparkling/itest/local/AirlinesWithWeatherDemo2Suite.scala new file mode 100644 index 000000000..ae2e3a477 --- /dev/null +++ b/examples/src/integTest/scala/water/sparkling/itest/local/AirlinesWithWeatherDemo2Suite.scala @@ -0,0 +1,27 @@ +package water.sparkling.itest.local + +import org.apache.spark.examples.h2o.AirlinesWithWeatherDemo2 +import org.junit.runner.RunWith +import org.scalatest.FunSuite +import org.scalatest.junit.JUnitRunner +import water.sparkling.itest.{IntegTestStopper, LocalTest, IntegTestHelper} + +@RunWith(classOf[JUnitRunner]) +class AirlinesWithWeatherDemo2Suite extends FunSuite with IntegTestHelper { + + test("Launch AirlinesWithWeatherDemo2", LocalTest) { + launch("water.sparkling.itest.local.AirlinesWithWeatherDemo2Test", + env { + sparkMaster("local-cluster[3,2,2048]") + conf("spark.executor.memory", "2g") + conf("spark.driver.memory", "2g") + } + ) + } +} + +object AirlinesWithWeatherDemo2Test extends IntegTestStopper{ + def main(args: Array[String]): Unit = exitOnException{ + AirlinesWithWeatherDemo2.main(args) + } +} diff --git a/examples/src/integTest/scala/water/sparkling/itest/local/AirlinesWithWeatherDemoSuite.scala b/examples/src/integTest/scala/water/sparkling/itest/local/AirlinesWithWeatherDemoSuite.scala new file mode 100644 index 000000000..7acb64570 --- /dev/null +++ b/examples/src/integTest/scala/water/sparkling/itest/local/AirlinesWithWeatherDemoSuite.scala @@ -0,0 +1,28 @@ +package water.sparkling.itest.local + +import org.apache.spark.examples.h2o.AirlinesWithWeatherDemo +import org.junit.runner.RunWith +import org.scalatest.FunSuite +import org.scalatest.junit.JUnitRunner +import water.sparkling.itest.{IntegTestStopper, LocalTest, IntegTestHelper} + + +@RunWith(classOf[JUnitRunner]) +class AirlinesWithWeatherDemoSuite extends FunSuite with IntegTestHelper { + + test("Launch AirlinesWithWeatherDemo", LocalTest) { + launch("water.sparkling.itest.local.AirlinesWithWeatherDemoTest", + env { + sparkMaster("local-cluster[3,2,2048]") + conf("spark.executor.memory", "2g") + conf("spark.driver.memory", "2g") + } + ) + } +} + +object AirlinesWithWeatherDemoTest extends IntegTestStopper{ + def main(args: Array[String]): Unit = exitOnException{ + AirlinesWithWeatherDemo.main(args) + } +} \ No newline at end of file diff --git a/examples/src/integTest/scala/water/sparkling/itest/local/ChicagoCrimeAppSmallSuite.scala b/examples/src/integTest/scala/water/sparkling/itest/local/ChicagoCrimeAppSmallSuite.scala new file mode 100644 index 000000000..8f61faf06 --- /dev/null +++ b/examples/src/integTest/scala/water/sparkling/itest/local/ChicagoCrimeAppSmallSuite.scala @@ -0,0 +1,27 @@ +package water.sparkling.itest.local + +import org.apache.spark.examples.h2o.ChicagoCrimeAppSmall +import org.junit.runner.RunWith +import org.scalatest.FunSuite +import org.scalatest.junit.JUnitRunner +import water.sparkling.itest.{IntegTestHelper, IntegTestStopper, LocalTest} + +@RunWith(classOf[JUnitRunner]) +class ChicagoCrimeAppSmallSuite extends FunSuite with IntegTestHelper { + + test("Launch Chicago Crime Demo", LocalTest) { + launch("water.sparkling.itest.local.ChicagoCrimeAppSmallTest", + env { + sparkMaster("local-cluster[3,2,3072]") + conf("spark.executor.memory", "3g") + conf("spark.driver.memory", "3g") + } + ) + } +} + +object ChicagoCrimeAppSmallTest extends IntegTestStopper{ + def main(args: Array[String]): Unit = exitOnException{ + ChicagoCrimeAppSmall.main(args) + } +} \ No newline at end of file diff --git a/examples/src/integTest/scala/water/sparkling/itest/local/CraigslistJobTitlesAppSuite.scala b/examples/src/integTest/scala/water/sparkling/itest/local/CraigslistJobTitlesAppSuite.scala new file mode 100644 index 000000000..9fb591d80 --- /dev/null +++ b/examples/src/integTest/scala/water/sparkling/itest/local/CraigslistJobTitlesAppSuite.scala @@ -0,0 +1,27 @@ +package water.sparkling.itest.local + +import org.apache.spark.examples.h2o.CraigslistJobTitlesApp +import org.junit.runner.RunWith +import org.scalatest.FunSuite +import org.scalatest.junit.JUnitRunner +import water.sparkling.itest.{IntegTestHelper, IntegTestStopper, LocalTest} + +@RunWith(classOf[JUnitRunner]) +class CraigslistJobTitlesAppSuite extends FunSuite with IntegTestHelper { + + test("Launch Craigslist App Demo", LocalTest) { + launch("water.sparkling.itest.local.CraigslistJobTitlesAppTest", + env { + sparkMaster("local-cluster[3,2,2048]") + conf("spark.executor.memory", "2g") + conf("spark.driver.memory", "2g") + } + ) + } +} + +object CraigslistJobTitlesAppTest extends IntegTestStopper{ + def main(args: Array[String]): Unit = exitOnException{ + CraigslistJobTitlesApp.main(args) + } +} \ No newline at end of file diff --git a/examples/src/integTest/scala/water/sparkling/itest/local/DeepLearningDemoSuite.scala b/examples/src/integTest/scala/water/sparkling/itest/local/DeepLearningDemoSuite.scala new file mode 100644 index 000000000..3c1fa5104 --- /dev/null +++ b/examples/src/integTest/scala/water/sparkling/itest/local/DeepLearningDemoSuite.scala @@ -0,0 +1,27 @@ +package water.sparkling.itest.local + +import org.apache.spark.examples.h2o.DeepLearningDemo +import org.junit.runner.RunWith +import org.scalatest.FunSuite +import org.scalatest.junit.JUnitRunner +import water.sparkling.itest.{IntegTestStopper, LocalTest, IntegTestHelper} + +@RunWith(classOf[JUnitRunner]) +class DeepLearningDemoSuite extends FunSuite with IntegTestHelper { + + test("Launch DeepLearnigDemo", LocalTest) { + launch("water.sparkling.itest.local.DeepLearningDemoTest", + env { + sparkMaster("local-cluster[3,2,2048]") + conf("spark.executor.memory", "2g") + conf("spark.driver.memory", "2g") + } + ) + } +} + +object DeepLearningDemoTest extends IntegTestStopper{ + def main(args: Array[String]): Unit = exitOnException{ + DeepLearningDemo.main(args) + } +} \ No newline at end of file diff --git a/examples/src/integTest/scala/water/sparkling/itest/local/DeepLearningDemoWithoutExtSuite.scala b/examples/src/integTest/scala/water/sparkling/itest/local/DeepLearningDemoWithoutExtSuite.scala new file mode 100644 index 000000000..c657bfd87 --- /dev/null +++ b/examples/src/integTest/scala/water/sparkling/itest/local/DeepLearningDemoWithoutExtSuite.scala @@ -0,0 +1,27 @@ +package water.sparkling.itest.local + +import org.apache.spark.examples.h2o.DeepLearningDemoWithoutExtension +import org.junit.runner.RunWith +import org.scalatest.FunSuite +import org.scalatest.junit.JUnitRunner +import water.sparkling.itest.{IntegTestStopper, IntegTestHelper, LocalTest} + +@RunWith(classOf[JUnitRunner]) +class DeepLearningDemoWithoutExtSuite extends FunSuite with IntegTestHelper { + + test("Launch DeepLearningDemoWithoutExtension", LocalTest) { + launch("water.sparkling.itest.local.DeepLearningDemoWithoutExtTest", + env { + sparkMaster("local-cluster[3,2,2048]") + conf("spark.executor.memory", "2g") + conf("spark.driver.memory", "2g") + } + ) + } +} + +object DeepLearningDemoWithoutExtTest extends IntegTestStopper{ + def main(args: Array[String]): Unit = exitOnException{ + DeepLearningDemoWithoutExtension.main(args) + } +} \ No newline at end of file diff --git a/examples/src/integTest/scala/water/sparkling/itest/local/HamOrSPamDemoSuite.scala b/examples/src/integTest/scala/water/sparkling/itest/local/HamOrSPamDemoSuite.scala new file mode 100644 index 000000000..570890612 --- /dev/null +++ b/examples/src/integTest/scala/water/sparkling/itest/local/HamOrSPamDemoSuite.scala @@ -0,0 +1,28 @@ +package water.sparkling.itest.local + +import org.apache.spark.examples.h2o.HamOrSpamDemo +import org.junit.runner.RunWith +import org.scalatest.FunSuite +import org.scalatest.junit.JUnitRunner +import water.sparkling.itest +import water.sparkling.itest.{IntegTestHelper, LocalTest} + +@RunWith(classOf[JUnitRunner]) +class HamOrSpamDemoSuite extends FunSuite with IntegTestHelper { + + test("Launch HamOrSpamDemo", LocalTest) { + launch("water.sparkling.itest.local.HamOrSpamDemoTest", + env { + sparkMaster("local-cluster[3,2,2048]") + conf("spark.executor.memory", "2g") + conf("spark.driver.memory", "2g") + } + ) + } +} + +object HamOrSpamDemoTest extends itest.IntegTestStopper{ + def main(args: Array[String]): Unit = exitOnException{ + HamOrSpamDemo.main(args) + } +} \ No newline at end of file diff --git a/examples/src/integTest/scala/water/sparkling/itest/local/ProstateDemoSuite.scala b/examples/src/integTest/scala/water/sparkling/itest/local/ProstateDemoSuite.scala new file mode 100644 index 000000000..8d0e3c41e --- /dev/null +++ b/examples/src/integTest/scala/water/sparkling/itest/local/ProstateDemoSuite.scala @@ -0,0 +1,27 @@ +package water.sparkling.itest.local + +import org.apache.spark.examples.h2o.ProstateDemo +import org.junit.runner.RunWith +import org.scalatest.FunSuite +import org.scalatest.junit.JUnitRunner +import water.sparkling.itest.{IntegTestHelper, IntegTestStopper, LocalTest} + +@RunWith(classOf[JUnitRunner]) +class ProstateDemoSuite extends FunSuite with IntegTestHelper { + + test("Launch ProstateDemo", LocalTest) { + launch("water.sparkling.itest.local.ProstateDemoTest", + env { + sparkMaster("local-cluster[3,2,2048]") + conf("spark.executor.memory", "2g") + conf("spark.driver.memory", "2g") + } + ) + } +} + +object ProstateDemoTest extends IntegTestStopper{ + def main(args: Array[String]): Unit = exitOnException{ + ProstateDemo.main(args) + } +} \ No newline at end of file diff --git a/examples/src/integTest/scala/water/sparkling/itest/local/PubDev457Suite.scala b/examples/src/integTest/scala/water/sparkling/itest/local/PubDev457Suite.scala index e7212bd96..b2135af54 100644 --- a/examples/src/integTest/scala/water/sparkling/itest/local/PubDev457Suite.scala +++ b/examples/src/integTest/scala/water/sparkling/itest/local/PubDev457Suite.scala @@ -8,8 +8,9 @@ import org.apache.spark.sql.SQLContext import org.junit.runner.RunWith import org.scalatest.FunSuite import org.scalatest.junit.JUnitRunner -import water.sparkling.itest.{LocalTest, IntegTestHelper} import water.support.SparkContextSupport +import water.sparkling.itest.{IntegTestStopper, LocalTest, IntegTestHelper} + /** * PUBDEV-457 test suite. @@ -28,13 +29,13 @@ class PubDev457Suite extends FunSuite with IntegTestHelper { } } -object PubDev457Test extends SparkContextSupport { +object PubDev457Test extends SparkContextSupport with IntegTestStopper{ case class LabeledDocument(id: Long, text: String, label: Double) case class Document(id: Long, text: String) - def main(args: Array[String]): Unit = { + def main(args: Array[String]): Unit = exitOnException{ val conf = configure("PUBDEV-457") val sc = new SparkContext(conf) val h2oContext = H2OContext.getOrCreate(sc) diff --git a/examples/src/integTest/scala/water/sparkling/itest/local/PubDev928Suite.scala b/examples/src/integTest/scala/water/sparkling/itest/local/PubDev928Suite.scala index 63975f081..ad331bd9a 100644 --- a/examples/src/integTest/scala/water/sparkling/itest/local/PubDev928Suite.scala +++ b/examples/src/integTest/scala/water/sparkling/itest/local/PubDev928Suite.scala @@ -3,16 +3,17 @@ package water.sparkling.itest.local import hex.deeplearning.DeepLearning import hex.deeplearning.DeepLearningModel.DeepLearningParameters import org.apache.spark.SparkContext -import org.apache.spark.examples.h2o.Airlines + +import org.apache.spark.examples.h2o.AirlinesParse import org.apache.spark.h2o.H2OContext -import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLContext import org.junit.runner.RunWith import org.scalatest.FunSuite import org.scalatest.junit.JUnitRunner import water.fvec.H2OFrame -import water.sparkling.itest.{LocalTest, IntegTestHelper} import water.support.SparkContextSupport +import water.sparkling.itest.{IntegTestStopper, LocalTest, IntegTestHelper} + /** * PUBDEV-928 test suite. @@ -33,15 +34,14 @@ class PubDev928Suite extends FunSuite with IntegTestHelper { } } -object PubDev928Test extends SparkContextSupport { +object PubDev928Test extends SparkContextSupport with IntegTestStopper { - def main(args: Array[String]): Unit = { + def main(args: Array[String]): Unit = exitOnException{ val conf = configure("PUBDEV-928") val sc = new SparkContext(conf) val h2oContext = H2OContext.getOrCreate(sc) - import h2oContext._ import h2oContext.implicits._ - val sqlContext = new SQLContext(sc) + implicit val sqlContext = new SQLContext(sc) import sqlContext.implicits._ val airlinesData = new H2OFrame(new java.io.File("examples/smalldata/allyears2k_headers.csv.gz")) @@ -49,7 +49,7 @@ object PubDev928Test extends SparkContextSupport { // We need to explicitly repartition data to 12 chunks/partitions since H2O parser handling // partitioning dynamically based on available number of CPUs println("Number of chunks before query: " + airlinesData.anyVec().nChunks()) - val airlinesTable: RDD[Airlines] = asRDD[Airlines](airlinesData) + val airlinesTable = h2oContext.asDataFrame(airlinesData).map(row => AirlinesParse(row)) airlinesTable.toDF.registerTempTable("airlinesTable") val query = "SELECT * FROM airlinesTable WHERE Dest LIKE 'SFO'" @@ -57,7 +57,7 @@ object PubDev928Test extends SparkContextSupport { val queryResult = sqlContext.sql(query) val partitionNumbers = queryResult.count().asInstanceOf[Int] + 1 val result: H2OFrame = h2oContext.asH2OFrame(queryResult.repartition(partitionNumbers), "flightTable") - println("Number of partitions in query result: " + queryResult.rdd.partitions.size) + println("Number of partitions in query result: " + queryResult.rdd.partitions.length) println("Number of chunks in query result" + result.anyVec().nChunks()) val train: H2OFrame = result('Year, 'Month, 'DayofMonth, 'DayOfWeek, 'CRSDepTime, 'CRSArrTime, diff --git a/examples/src/integTest/scala/water/sparkling/itest/local/RegularExamplesSuite.scala b/examples/src/integTest/scala/water/sparkling/itest/local/RegularExamplesSuite.scala deleted file mode 100644 index aea00f93b..000000000 --- a/examples/src/integTest/scala/water/sparkling/itest/local/RegularExamplesSuite.scala +++ /dev/null @@ -1,129 +0,0 @@ -package water.sparkling.itest.local - -import org.junit.runner.RunWith -import org.scalatest.FunSuite -import org.scalatest.junit.JUnitRunner -import water.sparkling.itest.{IntegTestHelper, LocalTest} - -/** - * Test launcher for examples. - * - * We need to provide a separated suite for each H2O example - * since H2O is not able to restart its context between - * runs in the same JVM. - * - * Better solution would be use test tags to filter - * them based on basic properties. - */ - -@RunWith(classOf[JUnitRunner]) -class AirlinesWithWeatherDemoSuite extends FunSuite with IntegTestHelper { - - test("Launch AirlinesWithWeatherDemo", LocalTest) { - launch("org.apache.spark.examples.h2o.AirlinesWithWeatherDemo", - env { - sparkMaster("local-cluster[3,2,2048]") - conf("spark.executor.memory", "2g") - conf("spark.driver.memory", "2g") - } - ) - } -} - -@RunWith(classOf[JUnitRunner]) -class AirlinesWithWeatherDemo2Suite extends FunSuite with IntegTestHelper { - - test("Launch AirlinesWithWeatherDemo2", LocalTest) { - launch("org.apache.spark.examples.h2o.AirlinesWithWeatherDemo2", - env { - sparkMaster("local-cluster[3,2,2048]") - conf("spark.executor.memory", "2g") - conf("spark.driver.memory", "2g") - } - ) - } -} - -@RunWith(classOf[JUnitRunner]) -class DeepLearnigDemoSuite extends FunSuite with IntegTestHelper { - - test("Launch DeepLearnigDemo", LocalTest) { - launch("org.apache.spark.examples.h2o.DeepLearningDemo", - env { - sparkMaster("local-cluster[3,2,2048]") - conf("spark.executor.memory", "2g") - conf("spark.driver.memory", "2g") - } - ) - } -} - -@RunWith(classOf[JUnitRunner]) -class DeepLearningDemoWithoutExtensionSuite extends FunSuite with IntegTestHelper { - - test("Launch DeepLearningDemoWithoutExtension", LocalTest) { - launch("org.apache.spark.examples.h2o.DeepLearningDemoWithoutExtension", - env { - sparkMaster("local-cluster[3,2,2048]") - conf("spark.executor.memory", "2g") - conf("spark.driver.memory", "2g") - } - ) - } -} - -@RunWith(classOf[JUnitRunner]) -class ProstateDemoSuite extends FunSuite with IntegTestHelper { - - test("Launch ProstateDemo", LocalTest) { - launch("org.apache.spark.examples.h2o.ProstateDemo", - env { - sparkMaster("local-cluster[3,2,2048]") - conf("spark.executor.memory", "2g") - conf("spark.driver.memory", "2g") - } - ) - } -} - -@RunWith(classOf[JUnitRunner]) -class HamOrSpamDemoSuite extends FunSuite with IntegTestHelper { - - test("Launch HamOrSpamDemo", LocalTest) { - launch("org.apache.spark.examples.h2o.HamOrSpamDemo", - env { - sparkMaster("local-cluster[3,2,2048]") - conf("spark.executor.memory", "2g") - conf("spark.driver.memory", "2g") - } - ) - } -} - -@RunWith(classOf[JUnitRunner]) -class ChicagoCrimeAppSmallSuite extends FunSuite with IntegTestHelper { - - test("Launch Chicago Crime Demo", LocalTest) { - launch("org.apache.spark.examples.h2o.ChicagoCrimeAppSmall", - env { - sparkMaster("local-cluster[3,2,3072]") - conf("spark.executor.memory", "3g") - conf("spark.driver.memory", "3g") - } - ) - } -} - -@RunWith(classOf[JUnitRunner]) -class CraigslistJobTitlesAppSuite extends FunSuite with IntegTestHelper { - - test("Launch Craigslist App Demo", LocalTest) { - launch("org.apache.spark.examples.h2o.CraigslistJobTitlesApp", - env { - sparkMaster("local-cluster[3,2,2048]") - conf("spark.executor.memory", "2g") - conf("spark.driver.memory", "2g") - } - ) - } -} diff --git a/examples/src/integTest/scala/water/sparkling/itest/standalone/HexDev100TestSuite.scala b/examples/src/integTest/scala/water/sparkling/itest/standalone/HexDev100TestSuite.scala index 64758857b..ca28289c0 100644 --- a/examples/src/integTest/scala/water/sparkling/itest/standalone/HexDev100TestSuite.scala +++ b/examples/src/integTest/scala/water/sparkling/itest/standalone/HexDev100TestSuite.scala @@ -8,7 +8,9 @@ import org.junit.runner.RunWith import org.scalatest.FunSuite import org.scalatest.junit.JUnitRunner import water.fvec.H2OFrame -import water.sparkling.itest.IntegTestHelper +import water.sparkling.itest.standalone.ParquetImportTest._ +import water.sparkling.itest.{IntegTestStopper, IntegTestHelper} + /** * Test for Jira Hex-Dev 100 : Import airlines data and run a host of classification models, @@ -31,9 +33,9 @@ class HexDev100TestSuite extends FunSuite with IntegTestHelper { } } -object HexDev100Test { +object HexDev100Test extends IntegTestStopper{ - def test(args: Array[String]): Unit = { + def main(args: Array[String]): Unit = exitOnException{ val conf = new SparkConf().setAppName("HexDev100Test") val sc = new SparkContext(conf) val h2oContext = H2OContext.getOrCreate(sc) diff --git a/examples/src/integTest/scala/water/sparkling/itest/standalone/HexDev62TestSuite.scala b/examples/src/integTest/scala/water/sparkling/itest/standalone/HexDev62TestSuite.scala index add55fbdf..bcb0d463d 100644 --- a/examples/src/integTest/scala/water/sparkling/itest/standalone/HexDev62TestSuite.scala +++ b/examples/src/integTest/scala/water/sparkling/itest/standalone/HexDev62TestSuite.scala @@ -7,7 +7,7 @@ import org.apache.spark.{SparkContext, SparkConf} import org.junit.runner.RunWith import org.scalatest.FunSuite import org.scalatest.junit.JUnitRunner -import water.sparkling.itest.IntegTestHelper +import water.sparkling.itest.{IntegTestStopper, IntegTestHelper} /** @@ -30,8 +30,9 @@ class HexDev62TestSuite extends FunSuite with IntegTestHelper { } } -object HexDev62Test { - def main(args: Array[String]): Unit = { +object HexDev62Test extends IntegTestStopper{ + + def main(args: Array[String]): Unit = exitOnException{ val conf = new SparkConf().setAppName("HexDev62Test") val sc = new SparkContext(conf) val h2oContext = H2OContext.getOrCreate(sc) diff --git a/examples/src/integTest/scala/water/sparkling/itest/standalone/HexDev64TestSuite.scala b/examples/src/integTest/scala/water/sparkling/itest/standalone/HexDev64TestSuite.scala index da439dc9a..d7a00625d 100644 --- a/examples/src/integTest/scala/water/sparkling/itest/standalone/HexDev64TestSuite.scala +++ b/examples/src/integTest/scala/water/sparkling/itest/standalone/HexDev64TestSuite.scala @@ -5,7 +5,7 @@ import org.apache.spark.{SparkConf, SparkContext} import org.junit.runner.RunWith import org.scalatest.FunSuite import org.scalatest.junit.JUnitRunner -import water.sparkling.itest.IntegTestHelper +import water.sparkling.itest.{IntegTestStopper, IntegTestHelper} /** @@ -28,8 +28,9 @@ class HexDev64TestSuite extends FunSuite with IntegTestHelper { } } -object HexDev64Test { - def main(args: Array[String]): Unit = { +object HexDev64Test extends IntegTestStopper{ + + def main(args: Array[String]): Unit = exitOnException{ val conf = new SparkConf().setAppName("HexDev64Test") val sc = new SparkContext(conf) val h2oContext = H2OContext.getOrCreate(sc) diff --git a/examples/src/integTest/scala/water/sparkling/itest/standalone/ParquetImportTestSuite.scala b/examples/src/integTest/scala/water/sparkling/itest/standalone/ParquetImportTestSuite.scala index 63d077591..4e286cd95 100644 --- a/examples/src/integTest/scala/water/sparkling/itest/standalone/ParquetImportTestSuite.scala +++ b/examples/src/integTest/scala/water/sparkling/itest/standalone/ParquetImportTestSuite.scala @@ -6,7 +6,7 @@ import org.apache.spark.{SparkContext, SparkConf} import org.junit.runner.RunWith import org.scalatest.FunSuite import org.scalatest.junit.JUnitRunner -import water.sparkling.itest.IntegTestHelper +import water.sparkling.itest.{IntegTestStopper, IntegTestHelper} /** * Test for Parquet Import : Save small airlines data as Parquet File, @@ -30,9 +30,9 @@ class ParquetImportTestSuite extends FunSuite with IntegTestHelper { } } -object ParquetImportTest { +object ParquetImportTest extends IntegTestStopper{ - def test(args: Array[String]): Unit = { + def main(args: Array[String]): Unit = exitOnException{ val conf = new SparkConf().setAppName("ParquetImportTest") // Launch H2O diff --git a/examples/src/integTest/scala/water/sparkling/itest/yarn/ChicagoCrimeTestSuite.scala b/examples/src/integTest/scala/water/sparkling/itest/yarn/ChicagoCrimeTestSuite.scala index 132ba1b6e..8a8cbebf3 100644 --- a/examples/src/integTest/scala/water/sparkling/itest/yarn/ChicagoCrimeTestSuite.scala +++ b/examples/src/integTest/scala/water/sparkling/itest/yarn/ChicagoCrimeTestSuite.scala @@ -7,8 +7,8 @@ import org.apache.spark.sql.SQLContext import org.junit.runner.RunWith import org.scalatest.FunSuite import org.scalatest.junit.JUnitRunner -import water.sparkling.itest.IntegTestHelper import water.support.SparkContextSupport +import water.sparkling.itest.{IntegTestStopper, IntegTestHelper} /** * Test following Alex's chicago crime demo. @@ -31,21 +31,9 @@ class ChicagoCrimeTestSuite extends FunSuite with IntegTestHelper { } } -object ChicagoCrimeTest extends SparkContextSupport { +object ChicagoCrimeTest extends SparkContextSupport with IntegTestStopper { - def main(args: Array[String]): Unit = { - try { - test(args) - } catch { - case t: Throwable => { - System.err.println(t.toString) - t.printStackTrace() - water.H2O.exit(-1) - } - } - } - - def test(args: Array[String]) { + def main(args: Array[String]): Unit = exitOnException{ val sc = new SparkContext(configure("ChicagoCrimeTest")) // SQL support val sqlContext = new SQLContext(sc) diff --git a/examples/src/integTest/scala/water/sparkling/itest/yarn/HexDev62TestSuite.scala b/examples/src/integTest/scala/water/sparkling/itest/yarn/HexDev62TestSuite.scala index 149125914..80505889c 100644 --- a/examples/src/integTest/scala/water/sparkling/itest/yarn/HexDev62TestSuite.scala +++ b/examples/src/integTest/scala/water/sparkling/itest/yarn/HexDev62TestSuite.scala @@ -7,7 +7,7 @@ import org.apache.spark.{SparkContext, SparkConf} import org.junit.runner.RunWith import org.scalatest.FunSuite import org.scalatest.junit.JUnitRunner -import water.sparkling.itest.IntegTestHelper +import water.sparkling.itest.{IntegTestStopper, IntegTestHelper} /** * Test suite for given JIRA bug. @@ -31,21 +31,9 @@ class HexDev62TestSuite extends FunSuite with IntegTestHelper { } } -object HexDev62Test { +object HexDev62Test extends IntegTestStopper{ - def main(args: Array[String]): Unit = { - try { - test(args) - } catch { - case t: Throwable => { - System.err.println(t.toString) - t.printStackTrace() - water.H2O.exit(-1) - } - } - } - - def test(args: Array[String]): Unit = { + def main(args: Array[String]): Unit = exitOnException{ val conf = new SparkConf().setAppName("HexDev62Test") val sc = new SparkContext(conf) val h2oContext = H2OContext.getOrCreate(sc) diff --git a/examples/src/integTest/scala/water/sparkling/itest/yarn/HexDev64TestSuite.scala b/examples/src/integTest/scala/water/sparkling/itest/yarn/HexDev64TestSuite.scala index 8af842d54..e94c610b1 100644 --- a/examples/src/integTest/scala/water/sparkling/itest/yarn/HexDev64TestSuite.scala +++ b/examples/src/integTest/scala/water/sparkling/itest/yarn/HexDev64TestSuite.scala @@ -5,7 +5,7 @@ import org.apache.spark.{SparkConf, SparkContext} import org.junit.runner.RunWith import org.scalatest.FunSuite import org.scalatest.junit.JUnitRunner -import water.sparkling.itest.IntegTestHelper +import water.sparkling.itest.{IntegTestStopper, IntegTestHelper} @RunWith(classOf[JUnitRunner]) class HexDev64TestSuite extends FunSuite with IntegTestHelper { @@ -25,21 +25,9 @@ class HexDev64TestSuite extends FunSuite with IntegTestHelper { } } -object HexDev64Test { +object HexDev64Test extends IntegTestStopper{ - def main(args: Array[String]): Unit = { - try { - test(args) - } catch { - case t: Throwable => { - System.err.println(t.toString) - t.printStackTrace() - water.H2O.exit(-1) - } - } - } - - def test(args: Array[String]): Unit = { + def main(args: Array[String]): Unit = exitOnException{ val conf = new SparkConf().setAppName("HexDev64Test") val sc = new SparkContext(conf) val h2oContext = H2OContext.getOrCreate(sc) diff --git a/examples/src/integTest/scala/water/sparkling/itest/yarn/KMeansTestSuite.scala b/examples/src/integTest/scala/water/sparkling/itest/yarn/KMeansTestSuite.scala index b1d4fcabb..ead6bbc76 100644 --- a/examples/src/integTest/scala/water/sparkling/itest/yarn/KMeansTestSuite.scala +++ b/examples/src/integTest/scala/water/sparkling/itest/yarn/KMeansTestSuite.scala @@ -9,7 +9,7 @@ import org.apache.spark.{SparkConf, SparkContext} import org.junit.runner.RunWith import org.scalatest.FunSuite import org.scalatest.junit.JUnitRunner -import water.sparkling.itest.{IntegTestHelper, YarnTest} +import water.sparkling.itest.{IntegTestStopper, IntegTestHelper, YarnTest} import water.util.Timer @RunWith(classOf[JUnitRunner]) @@ -35,21 +35,9 @@ class KMeansITestSuite extends FunSuite with IntegTestHelper { * Test runner loading large airlines data from YARN HDFS via H2O API * transforming them into RDD and launching MLlib K-means. */ -object KMeansITest { - - def main(args: Array[String]): Unit = { - try { - test(args) - } catch { - case t: Throwable => { - System.err.println(t.toString) - t.printStackTrace() - water.H2O.exit(-1) - } - } - } +object KMeansITest extends IntegTestStopper{ - def test(args: Array[String]): Unit = { + def main(args: Array[String]): Unit = exitOnException { val conf = new SparkConf().setAppName("KMeansITest") val sc = new SparkContext(conf) val h2oContext = H2OContext.getOrCreate(sc) diff --git a/examples/src/main/scala/org/apache/spark/examples/h2o/AirlinesWithWeatherDemo.scala b/examples/src/main/scala/org/apache/spark/examples/h2o/AirlinesWithWeatherDemo.scala index 1e390dc51..8dbc6ca7a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/h2o/AirlinesWithWeatherDemo.scala +++ b/examples/src/main/scala/org/apache/spark/examples/h2o/AirlinesWithWeatherDemo.scala @@ -37,6 +37,8 @@ object AirlinesWithWeatherDemo extends SparkContextSupport { // Create SparkContext to execute application on Spark cluster val sc = new SparkContext(conf) + implicit val sqlContext = SQLContext.getOrCreate(sc) + import sqlContext.implicits._ // import implicit conversions val h2oContext = H2OContext.getOrCreate(sc) import h2oContext._ import h2oContext.implicits._ @@ -54,15 +56,13 @@ object AirlinesWithWeatherDemo extends SparkContextSupport { // Use super-fast advanced H2O CSV parser !!! val airlinesData = new H2OFrame(new File(SparkFiles.get("allyears2k_headers.csv.gz"))) - val airlinesTable : RDD[Airlines] = asRDD[Airlines](airlinesData) + val airlinesTable = h2oContext.asDataFrame(airlinesData).map(row => AirlinesParse(row)) // Select flights only to ORD val flightsToORD = airlinesTable.filter(f => f.Dest==Some("ORD")) flightsToORD.count println(s"\nFlights to ORD: ${flightsToORD.count}\n") - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ // import implicit conversions flightsToORD.toDF.registerTempTable("FlightsToORD") weatherTable.toDF.registerTempTable("WeatherORD") diff --git a/examples/src/main/scala/org/apache/spark/examples/h2o/AirlinesWithWeatherDemo2.scala b/examples/src/main/scala/org/apache/spark/examples/h2o/AirlinesWithWeatherDemo2.scala index b1344cd16..13470a75e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/h2o/AirlinesWithWeatherDemo2.scala +++ b/examples/src/main/scala/org/apache/spark/examples/h2o/AirlinesWithWeatherDemo2.scala @@ -70,6 +70,8 @@ object AirlinesWithWeatherDemo2 extends SparkContextSupport { val conf: SparkConf = configure("Sparkling Water Meetup: Use Airlines and Weather Data for delay prediction") // Create SparkContext to execute application on Spark cluster val sc = new SparkContext(conf) + implicit val sqlContext = SQLContext.getOrCreate(sc) + import sqlContext.implicits._ // import implicit conversions val h2oContext = H2OContext.getOrCreate(sc) import h2oContext._ import h2oContext.implicits._ @@ -87,15 +89,13 @@ object AirlinesWithWeatherDemo2 extends SparkContextSupport { // Use super-fast advanced H2O CSV parser !!! val airlinesData = new H2OFrame(new File(SparkFiles.get("year2005.csv.gz"))) - val airlinesTable : RDD[Airlines] = asRDD[Airlines](airlinesData) + val airlinesTable = h2oContext.asDataFrame(airlinesData).map(row => AirlinesParse(row)) // Select flights only to ORD val flightsToORD = airlinesTable.filter(f => f.Dest==Some("ORD")) flightsToORD.count println(s"\nFlights to ORD: ${flightsToORD.count}\n") - implicit val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ // import implicit conversions flightsToORD.toDF.registerTempTable("FlightsToORD") weatherTable.toDF.registerTempTable("WeatherORD") diff --git a/examples/src/main/scala/org/apache/spark/examples/h2o/DeepLearningDemo.scala b/examples/src/main/scala/org/apache/spark/examples/h2o/DeepLearningDemo.scala index 2a6159bb3..ac392391f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/h2o/DeepLearningDemo.scala +++ b/examples/src/main/scala/org/apache/spark/examples/h2o/DeepLearningDemo.scala @@ -36,6 +36,9 @@ object DeepLearningDemo extends SparkContextSupport { val sc = new SparkContext(conf) addFiles(sc, absPath("examples/smalldata/allyears2k_headers.csv.gz")) + implicit val sqlContext = SQLContext.getOrCreate(sc) + import sqlContext.implicits._ // import implicit conversions + // Run H2O cluster inside Spark cluster val h2oContext = H2OContext.getOrCreate(sc) import h2oContext._ @@ -49,16 +52,13 @@ object DeepLearningDemo extends SparkContextSupport { // // Use H2O to RDD transformation // - val airlinesTable : RDD[Airlines] = asRDD[Airlines](airlinesData) + val airlinesTable = h2oContext.asDataFrame(airlinesData).map(row => AirlinesParse(row)) println(s"\n===> Number of all flights via RDD#count call: ${airlinesTable.count()}\n") println(s"\n===> Number of all flights via H2O#Frame#count: ${airlinesData.numRows()}\n") // // Filter data with help of Spark SQL // - - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ // import implicit conversions airlinesTable.toDF.registerTempTable("airlinesTable") // Select only interesting columns and flights with destination in SFO diff --git a/examples/src/main/scala/org/apache/spark/examples/h2o/DeepLearningDemoWithoutExtension.scala b/examples/src/main/scala/org/apache/spark/examples/h2o/DeepLearningDemoWithoutExtension.scala index 310afbac2..b64dc51a7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/h2o/DeepLearningDemoWithoutExtension.scala +++ b/examples/src/main/scala/org/apache/spark/examples/h2o/DeepLearningDemoWithoutExtension.scala @@ -38,6 +38,8 @@ object DeepLearningDemoWithoutExtension extends SparkContextSupport { val sc = new SparkContext(conf) addFiles(sc, absPath("examples/smalldata/allyears2k_headers.csv.gz")) + implicit val sqlContext = SQLContext.getOrCreate(sc) + import sqlContext.implicits._ // import implicit conversions val h2oContext = H2OContext.getOrCreate(sc) import h2oContext._ import h2oContext.implicits._ @@ -50,7 +52,7 @@ object DeepLearningDemoWithoutExtension extends SparkContextSupport { // // Use H2O to RDD transformation // - val airlinesTable : RDD[Airlines] = asRDD[Airlines](airlinesData) + val airlinesTable = h2oContext.asDataFrame(airlinesData).map(row => AirlinesParse(row)) println(s"\n===> Number of all flights via RDD#count call: ${airlinesTable.count()}\n") println(s"\n===> Number of all flights via H2O#Frame#count: ${airlinesData.numRows()}\n") @@ -58,8 +60,6 @@ object DeepLearningDemoWithoutExtension extends SparkContextSupport { // Filter data with help of Spark SQL // - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ // import implicit conversions airlinesTable.toDF.registerTempTable("airlinesTable") // Select only interesting columns and flights with destination in SFO diff --git a/examples/src/main/scala/org/apache/spark/examples/h2o/Schemas.scala b/examples/src/main/scala/org/apache/spark/examples/h2o/Schemas.scala index acaae2156..4c8b490ee 100644 --- a/examples/src/main/scala/org/apache/spark/examples/h2o/Schemas.scala +++ b/examples/src/main/scala/org/apache/spark/examples/h2o/Schemas.scala @@ -17,6 +17,7 @@ package org.apache.spark.examples.h2o +import org.apache.spark.sql.Row import org.joda.time.{DateTimeZone, MutableDateTime} /** Prostate schema definition. */ @@ -128,6 +129,11 @@ class Airlines (val Year :Option[Int], /** A dummy csv parser for airlines dataset. */ object AirlinesParse extends Serializable { + + def apply(row: Row): Airlines = { + apply(row.mkString(",").split(",")) + } + def apply(row: Array[String]): Airlines = { import water.support.ParseSupport._ new Airlines(int (row( 0)), // Year diff --git a/examples/src/scriptsTest/scala/water/sparkling/scripts/ScriptTestHelper.scala b/examples/src/scriptsTest/scala/water/sparkling/scripts/ScriptTestHelper.scala index 13a8b2e64..5a7776923 100644 --- a/examples/src/scriptsTest/scala/water/sparkling/scripts/ScriptTestHelper.scala +++ b/examples/src/scriptsTest/scala/water/sparkling/scripts/ScriptTestHelper.scala @@ -3,8 +3,8 @@ package water.sparkling.scripts import java.io.File import org.apache.spark.repl.h2o.{CodeResults, H2OInterpreter} -import org.apache.spark.{SparkContext, SparkConf} -import org.scalatest.{Suite, BeforeAndAfterAll, FunSuite} +import org.apache.spark.{SparkConf, SparkContext} +import org.scalatest.{BeforeAndAfterAll, FunSuite, Suite} import scala.collection.immutable.HashMap import scala.collection.mutable.ListBuffer @@ -20,18 +20,18 @@ trait ScriptsTestHelper extends FunSuite with org.apache.spark.Logging with Befo super.beforeAll() } - override protected def afterAll(): Unit = { if(sc!=null){ sc.stop() } + super.afterAll() } def defaultConf: SparkConf = { val assemblyJar = sys.props.getOrElse("sparkling.assembly.jar", fail("The variable 'sparkling.assembly.jar' is not set! It should point to assembly jar file.")) val conf = new SparkConf().setAppName("Script testing") - .set("spark.repl.class.uri",H2OInterpreter.classServerUri) + .set("spark.repl.class.uri", H2OInterpreter.classServerUri) .set("spark.ext.h2o.repl.enabled","false") // disable repl in tests .set("spark.driver.extraJavaOptions", "-XX:MaxPermSize=384m") .set("spark.executor.extraJavaOptions", "-XX:MaxPermSize=384m") @@ -42,6 +42,7 @@ trait ScriptsTestHelper extends FunSuite with org.apache.spark.Logging with Befo conf } + private def launch(code: String, loop: H2OInterpreter, inspections: ScriptInspections): ScriptTestResult = { val testResult = new ScriptTestResult() val codeExecutionStatus = loop.runCode(code) @@ -57,7 +58,7 @@ trait ScriptsTestHelper extends FunSuite with org.apache.spark.Logging with Befo inspections.termsAndValues.foreach { termName => - testResult.addTermValue(termName,loop.valueOfTerm(termName).get.toString) + testResult.addTermValue(termName, loop.valueOfTerm(termName).get.toString) } testResult @@ -71,7 +72,7 @@ trait ScriptsTestHelper extends FunSuite with org.apache.spark.Logging with Befo val code = scala.io.Source.fromFile(sourceFile).mkString val loop = new H2OInterpreter(sc, sessionId = 1) - val res = launch(code,loop, inspections) + val res = launch(code, loop, inspections) loop.closeInterpreter() res } diff --git a/gradle/scala.gradle b/gradle/scala.gradle index 170901d43..5da8c08bc 100644 --- a/gradle/scala.gradle +++ b/gradle/scala.gradle @@ -1,4 +1,5 @@ apply plugin: 'scala' +apply from: "$rootDir/gradle/utils.gradle" // Activate Zinc compiler and configure scalac tasks.withType(ScalaCompile) { diff --git a/gradle/spark.gradle b/gradle/spark.gradle index 597bc0d45..558daec9f 100644 --- a/gradle/spark.gradle +++ b/gradle/spark.gradle @@ -94,9 +94,10 @@ task printTestEnv << { task printIntegTestEnv << { println("-Dspark.testing=true\n" + "-Dsparkling.assembly.jar=${project(":sparkling-water-assembly").configurations.shadow.artifacts.file.join(',')}\n" + - "-Dparkling.itest.jar=${integTestJar.archivePath}\n"+ + "-Dsparkling.itest.jar=${integTestJar.archivePath}\n"+ "-Dsparkling.pysparkling.egg=${project(":sparkling-water-py").configurations.eggs.artifacts.file.join(',')}\n"+ - "-Dsparkling.test.hdp.version=current") + "-Dsparkling.test.hdp.version=current\n"+ + "-DSPARK_HOME=${sparkHome}") } task printScriptsTestEnv << { diff --git a/ml/src/main/scala/org/apache/spark/ml/h2o/models/H2OParams.scala b/ml/src/main/scala/org/apache/spark/ml/h2o/models/H2OParams.scala index f5f0b2871..0aebc1b25 100644 --- a/ml/src/main/scala/org/apache/spark/ml/h2o/models/H2OParams.scala +++ b/ml/src/main/scala/org/apache/spark/ml/h2o/models/H2OParams.scala @@ -18,9 +18,9 @@ package org.apache.spark.ml.h2o.models import com.google.common.base.CaseFormat import hex.Model.Parameters -import org.apache.spark.h2o.ReflectionUtils._ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.h2o.H2OKeyParam +import org.apache.spark.h2o.utils.ReflectionUtils._ import water.fvec.Frame import scala.reflect.ClassTag diff --git a/py/build.gradle b/py/build.gradle index 35adb0959..640d1b047 100644 --- a/py/build.gradle +++ b/py/build.gradle @@ -68,9 +68,9 @@ or """) } - if(H2O_HOME != null && H2O_PYTHON_WHEEL !=null){ - logger.info("Both \"H2O_HOME\" and \"H2O_PYTHON_WHEEL\" properties are set. Using \"H2O_HOME\"!") - } + if(H2O_HOME != null && H2O_PYTHON_WHEEL !=null){ + logger.info("Both \"H2O_HOME\" and \"H2O_PYTHON_WHEEL\" properties are set. Using \"H2O_HOME\"!") + } } diff --git a/py/examples/notebooks/ChicagoCrimeDemo.ipynb b/py/examples/notebooks/ChicagoCrimeDemo.ipynb index 1eb9dac60..eb7fb71fb 100644 --- a/py/examples/notebooks/ChicagoCrimeDemo.ipynb +++ b/py/examples/notebooks/ChicagoCrimeDemo.ipynb @@ -3,9 +3,7 @@ { "cell_type": "code", "execution_count": null, - "metadata": { - "collapsed": true - }, + "metadata": {}, "outputs": [], "source": [ "#Initiate H2OContext on top of Spark\n", @@ -16,9 +14,7 @@ { "cell_type": "code", "execution_count": null, - "metadata": { - "collapsed": true - }, + "metadata": {}, "outputs": [], "source": [ "# This is just helper function returning relative path to data files within sparkling-water project directories\n", @@ -46,15 +42,13 @@ "# uploads file located on current node and distributes it to the H2O cluster)\n", "f_weather = h2o.upload_file(SparkFiles.get(chicagoAllWeather))\n", "f_census = h2o.upload_file(SparkFiles.get(chicagoCensus))\n", - "f_crimes = h2o.upload_file(SparkFiles.get(chicagoCrimes10k))" + "f_crimes = h2o.upload_file(SparkFiles.get(chicagoCrimes10k), col_types = {\"Date\": \"string\"})" ] }, { "cell_type": "code", "execution_count": null, - "metadata": { - "collapsed": true - }, + "metadata": {}, "outputs": [], "source": [ "f_weather.show()\n", @@ -65,9 +59,7 @@ { "cell_type": "code", "execution_count": null, - "metadata": { - "collapsed": true - }, + "metadata": {}, "outputs": [], "source": [ "# Set time zone to UTC for date manipulation\n", @@ -77,9 +69,7 @@ { "cell_type": "code", "execution_count": null, - "metadata": { - "collapsed": true - }, + "metadata": {}, "outputs": [], "source": [ "# Transform weather table\n", @@ -90,9 +80,7 @@ { "cell_type": "code", "execution_count": null, - "metadata": { - "collapsed": true - }, + "metadata": {}, "outputs": [], "source": [ "# Transform census table\n", @@ -107,9 +95,7 @@ { "cell_type": "code", "execution_count": null, - "metadata": { - "collapsed": true - }, + "metadata": {}, "outputs": [], "source": [ "# Transform crimes table\n", @@ -145,9 +131,7 @@ { "cell_type": "code", "execution_count": null, - "metadata": { - "collapsed": true - }, + "metadata": {}, "outputs": [], "source": [ "# Expose H2O frames as Spark DataFrame\n", @@ -160,9 +144,7 @@ { "cell_type": "code", "execution_count": null, - "metadata": { - "collapsed": true - }, + "metadata": {}, "outputs": [], "source": [ "df_weather.show()" @@ -171,9 +153,7 @@ { "cell_type": "code", "execution_count": null, - "metadata": { - "collapsed": true - }, + "metadata": {}, "outputs": [], "source": [ "# Use Spark SQL to join datasets\n", @@ -202,9 +182,7 @@ { "cell_type": "code", "execution_count": null, - "metadata": { - "collapsed": true - }, + "metadata": {}, "outputs": [], "source": [ "crimeWithWeather.show()" @@ -213,9 +191,7 @@ { "cell_type": "code", "execution_count": null, - "metadata": { - "collapsed": true - }, + "metadata": {}, "outputs": [], "source": [ "# Publish Spark DataFrame as H2OFrame with given name\n", @@ -225,9 +201,7 @@ { "cell_type": "code", "execution_count": null, - "metadata": { - "collapsed": true - }, + "metadata": {}, "outputs": [], "source": [ "# Transform selected String columns to categoricals\n", @@ -242,9 +216,7 @@ { "cell_type": "code", "execution_count": null, - "metadata": { - "collapsed": true - }, + "metadata": {}, "outputs": [], "source": [ "# Split frame into two - we use one as the training frame and the second one as the validation frame\n", @@ -260,9 +232,7 @@ { "cell_type": "code", "execution_count": null, - "metadata": { - "collapsed": true - }, + "metadata": {}, "outputs": [], "source": [ "# Create and train GBM model\n", @@ -286,9 +256,7 @@ { "cell_type": "code", "execution_count": null, - "metadata": { - "collapsed": true - }, + "metadata": {}, "outputs": [], "source": [ "# Show GBM model performance\n", @@ -298,9 +266,7 @@ { "cell_type": "code", "execution_count": null, - "metadata": { - "collapsed": true - }, + "metadata": {}, "outputs": [], "source": [ "# Create and train deeplearning model\n", @@ -320,9 +286,7 @@ { "cell_type": "code", "execution_count": null, - "metadata": { - "collapsed": true - }, + "metadata": {}, "outputs": [], "source": [ "# Show deeplearning model performance\n", @@ -332,9 +296,7 @@ { "cell_type": "code", "execution_count": null, - "metadata": { - "collapsed": true - }, + "metadata": {}, "outputs": [], "source": [ "# Create crime class which is used as a data holder on which prediction is done\n", @@ -399,9 +361,7 @@ { "cell_type": "code", "execution_count": null, - "metadata": { - "collapsed": true - }, + "metadata": {}, "outputs": [], "source": [ "# Create crime examples\n", @@ -413,9 +373,7 @@ { "cell_type": "code", "execution_count": null, - "metadata": { - "collapsed": true - }, + "metadata": {}, "outputs": [], "source": [ "# For given crime and model return probability of crime.\n", @@ -456,7 +414,7 @@ "language_info": { "codemirror_mode": { "name": "ipython", - "version": 2 + "version": 2.0 }, "file_extension": ".py", "mimetype": "text/x-python", @@ -468,4 +426,4 @@ }, "nbformat": 4, "nbformat_minor": 0 -} +} \ No newline at end of file diff --git a/py/examples/scripts/ChicagoCrimeDemo.py b/py/examples/scripts/ChicagoCrimeDemo.py index dcf1d24a1..1dbdb56e5 100644 --- a/py/examples/scripts/ChicagoCrimeDemo.py +++ b/py/examples/scripts/ChicagoCrimeDemo.py @@ -89,7 +89,7 @@ def _locate(example_name): # SQL support sqlContext = SQLContext.getOrCreate(sc) # Start H2O services -h2oContext = H2OContext(sc).start() +h2oContext = H2OContext.getOrCreate(sc) # Define file names chicagoAllWeather = "chicagoAllWeather.csv" chicagoCensus = "chicagoCensus.csv" @@ -106,7 +106,7 @@ def _locate(example_name): # located on current node and distributes it to the H2O cluster) f_weather = h2o.upload_file(SparkFiles.get(chicagoAllWeather)) f_census = h2o.upload_file(SparkFiles.get(chicagoCensus)) -f_crimes = h2o.upload_file(SparkFiles.get(chicagoCrimes10k)) +f_crimes = h2o.upload_file(SparkFiles.get(chicagoCrimes10k), col_types = {"Date": "string"}) # Transform weather table diff --git a/py/examples/scripts/H2OContextInitDemo.py b/py/examples/scripts/H2OContextInitDemo.py index 8b00ee70d..ceb8dc5d1 100644 --- a/py/examples/scripts/H2OContextInitDemo.py +++ b/py/examples/scripts/H2OContextInitDemo.py @@ -9,7 +9,7 @@ sqlContext = SQLContext(sc) # initiate H2OContext -hc = H2OContext(sc).start() +hc = H2OContext.getOrCreate(sc) # stop H2O and Spark services h2o.shutdown(prompt=False) diff --git a/py/pysparkling/__init__.py b/py/pysparkling/__init__.py index f656b0e39..1cc84595a 100644 --- a/py/pysparkling/__init__.py +++ b/py/pysparkling/__init__.py @@ -9,6 +9,7 @@ # set imports from this project which will be available when the module is imported from pysparkling.context import H2OContext +from pysparkling.conf import H2OConf # set what is meant by * packages in statement from foo import * -__all__ = ["H2OContext"] +__all__ = ["H2OContext", "H2OConf"] diff --git a/py/pysparkling/conf.py b/py/pysparkling/conf.py new file mode 100644 index 000000000..02668ec39 --- /dev/null +++ b/py/pysparkling/conf.py @@ -0,0 +1,196 @@ + +from pysparkling.initializer import Initializer + +class H2OConf(object): + def __init__(self, spark_context): + try: + Initializer.load_sparkling_jar(spark_context) + self._do_init(spark_context) + except: + raise + + + def _do_init(self, spark_context): + self._sc = spark_context + jvm = self._sc._jvm + gw = self._sc._gateway + jsc = self._sc._jsc + conf_klazz = jvm.java.lang.Thread.currentThread().getContextClassLoader().loadClass("org.apache.spark.h2o.H2OConf") + # Find constructor with right arguments + constructor_def = gw.new_array(jvm.Class, 1) + constructor_def[0] = jsc.getClass() + jconf_constructor = conf_klazz.getConstructor(constructor_def) + constructor_params = gw.new_array(jvm.Object, 1) + constructor_params[0] = jsc + # Create instance of H2OConf class + self._jconf = jconf_constructor.newInstance(constructor_params) + + def _get_option(self, option): + if option.isDefined(): + return option.get() + else: + return None + + # setters for most common properties + def set_cloud_name(self, cloud_name): + self._jconf.setCloudName(cloud_name) + return self + + def set_client_ip(self, ip): + self._jconf.setClientIP(ip) + return self + + def set_flatfile_path(self, flatfile_path): + self._jconf.setFlatFilePath(flatfile_path) + return self + + # getters + def cloud_name(self): + return self._get_option(self._jconf.cloudName()) + + def num_H2O_Workers(self): + return self._get_option(self._jconf.numH2OWorkers()) + + def use_flatfile(self): + return self._jconf.useFlatFile() + + def node_base_port(self): + return self._jconf.nodeBasePort() + + def cloud_timeout(self): + return self._jconf.cloudTimeout() + + def drdd_mul_factor(self): + return self._jconf.drddMulFactor() + + def num_rdd_retries(self): + return self._jconf.numRddRetries() + + def default_cloud_size(self): + return self._jconf.defaultCloudSize() + + def h2o_node_log_level(self): + return self._jconf.h2oNodeLogLevel() + + def h2o_node_log_dir(self): + return self._jconf.h2oNodeLogDir() + + def node_iced_dir(self): + return self._get_option(self._jconf.nodeIcedDir()) + + def subseq_tries(self): + return self._jconf.subseqTries() + + + def client_ip(self): + return self._get_option(self._jconf.clientIp()) + + def client_base_port(self): + return self._jconf.clientBasePort() + + def h2o_client_log_level(self): + return self._jconf.h2oClientLogLevel() + + def h2o_client_log_dir(self): + return self._jconf.h2oClientLogDir() + + def client_network_mask(self): + return self._get_option(self._jconf.clientNetworkMask()) + + def node_network_mask(self): + return self._get_option(self._jconf.nodeNetworkMask()) + + def nthreads(self): + return self._jconf.nthreads() + + def disable_ga(self): + return self._jconf.disableGA() + + def client_web_port(self): + return self._jconf.clientWebPort() + + def client_iced_dir(self): + return self._get_option(self._jconf.clientIcedDir()) + + def jks(self): + return self._get_option(self._jconf.jks()) + + def jks_pass(self): + return self._get_option(self._jconf.jksPass()) + + def hash_login(self): + return self._jconf.hashLogin() + + def ldap_login(self): + return self._jconf.ldapLogin() + + def login_conf(self): + return self._get_option(self._jconf.loginConf()) + + def user_name(self): + return self._get_option(self._jconf.userName()) + + def scala_int_default_num(self): + return self._jconf.scalaIntDefaultNum() + + def is_h2o_repl_enabled(self): + return self._jconf.isH2OReplEnabled() + + def is_cluster_topology_listener_enabled(self): + return self._jconf.isClusterTopologyListenerEnabled() + + def is_spark_version_check_enabled(self): + return self._jconf.isSparkVersionCheckEnabled() + + + def set(self, key, value): + self._jconf.set(key, value) + return self + + def remove(self, key): + self._jconf.remove(key) + return self + + def contains(self, key): + return self._jconf.contains(key) + + + def get(self, key): + """ + Get a parameter; throws a NoSuchElementException if it's not set + """ + return self._jconf.get(key) + + + def get(self, key, defaultValue): + """ + Get a parameter, falling back to a default if not set + """ + return self._jconf.get(key, defaultValue) + + + + def get_all(self): + """ + Get all parameters as a list of pairs + :return: list_of_configurations: List of pairs containing configurations + """ + python_conf = [] + all = self._jconf.getAll() + for conf in all: + python_conf.append((conf._1(),conf._2())) + return python_conf + + + def set_all(self, list_of_configurations): + """ + Set multiple parameters together + :param list_of_configurations: List of pairs containing configurations + :return: this H2O configuration + """ + for conf in list_of_configurations: + self._jconf.set(conf[0], conf[1]) + self + + def __str__(self): + return self._jconf.toString() diff --git a/py/pysparkling/context.py b/py/pysparkling/context.py index cc92ed0bc..1b250b39b 100644 --- a/py/pysparkling/context.py +++ b/py/pysparkling/context.py @@ -3,10 +3,11 @@ from pyspark.rdd import RDD from pyspark.sql import SQLContext from h2o.frame import H2OFrame +from pysparkling.initializer import Initializer +from pysparkling.conf import H2OConf import h2o -from pysparkling.utils import FrameConversions as fc +from pysparkling.conversions import FrameConversions as fc import warnings -from pkg_resources import resource_filename def _monkey_patch_H2OFrame(hc): @staticmethod @@ -58,42 +59,50 @@ def _get_first(rdd): return rdd.first() -def get_sw_jar(): - return resource_filename("sparkling_water", 'sparkling_water_assembly.jar') + class H2OContext(object): - def __init__(self, sparkContext): + def __init__(self, spark_context): + """ + This constructor is used just to initialize the environment. It does not start H2OContext. + To start H2OContext use one of the getOrCreate methods. This constructor is internally used in those methods + """ try: - self._do_init(sparkContext) - # Hack H2OFrame from h2o package + self.__do_init(spark_context) _monkey_patch_H2OFrame(self) + # loads sparkling water jar only if it hasn't been already loaded + Initializer.load_sparkling_jar(spark_context) + except: raise - def _do_init(self, sparkContext): - self._sc = sparkContext - # Add Sparkling water assembly JAR to driver - url = self._sc._jvm.java.net.URL("file://"+get_sw_jar()) - # Assuming that context class loader is always instance of URLClassLoader ( which should be always true) - cl = self._sc._jvm.Thread.currentThread().getContextClassLoader() + def __do_init(self, spark_context): + self._sc = spark_context + # do not instantiate SQL Context when already one exists + self._jsql_context = self._sc._jvm.SQLContext.getOrCreate(self._sc._jsc.sc()) + self._sql_context = SQLContext(spark_context, self._jsql_context) + self._jsc = self._sc._jsc + self._jvm = self._sc._jvm + self._gw = self._sc._gateway - # explicitly check if we run on databricks cloud since there we must add the jar to the parent of context class loader - if cl.getClass().getName()=='com.databricks.backend.daemon.driver.DriverLocal$DriverLocalClassLoader': - cl.getParent().getParent().addURL(url) - else: - cl.addURL(url) + @staticmethod + def getOrCreate(spark_context, conf = None): + """ + Get existing or create new H2OContext based on provided H2O configuration. If the conf parameter is set then + configuration from it is used. Otherwise the configuration properties passed to Sparkling Water are used. + If the values are not found the default values are used in most of the cases. The default cluster mode + is internal, ie. spark.ext.h2o.external.cluster.mode=false - # Add Sparkling water assembly JAR to executors - self._sc._jsc.addJar(get_sw_jar()) + param - Spark Context + returns H2O Context + """ + h2o_context = H2OContext(spark_context) - # do not instantiate sqlContext when already one exists - self._jsqlContext = self._sc._jvm.SQLContext.getOrCreate(self._sc._jsc.sc()) - self._sqlContext = SQLContext(sparkContext,self._jsqlContext) - self._jsc = sparkContext._jsc - self._jvm = sparkContext._jvm - self._gw = sparkContext._gateway + jvm = h2o_context._jvm # JVM + gw = h2o_context._gw # Py4J Gateway + jsc = h2o_context._jsc # JavaSparkContext # Imports Sparkling Water into current JVM view # We cannot use directly Py4j to import Sparkling Water packages @@ -101,31 +110,28 @@ def _do_init(self, sparkContext): # because of https://issues.apache.org/jira/browse/SPARK-5185 # So lets load class directly via classloader # This is finally fixed in Spark 2.0 ( along with other related issues) - jvm = self._jvm - sc = self._sc - gw = self._gw - jhc_klazz = self._jvm.java.lang.Thread.currentThread().getContextClassLoader().loadClass("org.apache.spark.h2o.H2OContext") - # Find ctor with right spark context - jctor_def = gw.new_array(jvm.Class, 1) - jctor_def[0] = sc._jsc.getClass() - jhc_ctor = jhc_klazz.getConstructor(jctor_def) - jctor_params = gw.new_array(jvm.Object, 1) - jctor_params[0] = sc._jsc - # Create instance of class - self._jhc = jhc_ctor.newInstance(jctor_params) - - def start(self): - """ - Start H2OContext. - It initializes H2O services on each node in Spark cluster and creates - H2O python client. - """ - self._jhc.start() - self._client_ip = self._jhc.h2oLocalClientIp() - self._client_port = self._jhc.h2oLocalClientPort() - h2o.init(ip=self._client_ip, port=self._client_port,start_h2o=False, strict_version_check=False) - return self + # Call the corresponding getOrCreate method + jhc_klazz = jvm.java.lang.Thread.currentThread().getContextClassLoader().loadClass("org.apache.spark.h2o.JavaH2OContext") + conf_klazz = jvm.java.lang.Thread.currentThread().getContextClassLoader().loadClass("org.apache.spark.h2o.H2OConf") + method_def = gw.new_array(jvm.Class, 2) + method_def[0] = jsc.getClass() + method_def[1] = conf_klazz + method = jhc_klazz.getMethod("getOrCreate", method_def) + method_params = gw.new_array(jvm.Object, 2) + method_params[0] = jsc + if conf is not None: + selected_conf = conf + else: + selected_conf = H2OConf(spark_context) + method_params[1] = selected_conf._jconf + jhc = method.invoke(None, method_params) + h2o_context._jhc = jhc + h2o_context._conf = selected_conf + h2o_context._client_ip = jhc.h2oLocalClientIp() + h2o_context._client_port = jhc.h2oLocalClientPort() + h2o.init(ip=h2o_context._client_ip, port=h2o_context._client_port, start_h2o=False, strict_version_check=False) + return h2o_context def stop(self): warnings.warn("H2OContext stopping is not yet supported...") @@ -141,6 +147,9 @@ def __repr__(self): def show(self): print self + def get_conf(self): + return self._conf + def as_spark_frame(self, h2o_frame, copy_metadata = True): """ Transforms given H2OFrame to Spark DataFrame @@ -148,15 +157,16 @@ def as_spark_frame(self, h2o_frame, copy_metadata = True): Parameters ---------- h2o_frame : H2OFrame + copy_metadata: Bool = True Returns ------- Spark DataFrame """ - if isinstance(h2o_frame,H2OFrame): + if isinstance(h2o_frame, H2OFrame): j_h2o_frame = h2o_frame.get_java_h2o_frame() - jdf = self._jhc.asDataFrame(j_h2o_frame, copy_metadata, self._jsqlContext) - return DataFrame(jdf,self._sqlContext) + jdf = self._jhc.asDataFrame(j_h2o_frame, copy_metadata, self._jsql_context) + return DataFrame(jdf,self._sql_context) def as_h2o_frame(self, dataframe, framename = None): """ diff --git a/py/pysparkling/utils.py b/py/pysparkling/conversions.py similarity index 73% rename from py/pysparkling/utils.py rename to py/pysparkling/conversions.py index 483db0ab3..0813f5f4a 100644 --- a/py/pysparkling/utils.py +++ b/py/pysparkling/conversions.py @@ -4,53 +4,53 @@ class FrameConversions: @staticmethod - def _as_h2o_frame_from_RDD_String(h2oContext, rdd, framename): - j_h2o_frame = h2oContext._jhc.asH2OFrameFromRDDString(rdd._to_java_object_rdd(), framename) + def _as_h2o_frame_from_RDD_String(h2oContext, rdd, frame_name): + j_h2o_frame = h2oContext._jhc.asH2OFrameFromRDDString(rdd._to_java_object_rdd(), frame_name) j_h2o_frame_key = j_h2o_frame.key() return H2OFrame.from_java_h2o_frame(j_h2o_frame,j_h2o_frame_key) @staticmethod - def _as_h2o_frame_from_RDD_Bool(h2oContext, rdd, framename): - j_h2o_frame = h2oContext._jhc.asH2OFrameFromRDDBool(rdd._to_java_object_rdd(), framename) + def _as_h2o_frame_from_RDD_Bool(h2oContext, rdd, frame_name): + j_h2o_frame = h2oContext._jhc.asH2OFrameFromRDDBool(rdd._to_java_object_rdd(), frame_name) j_h2o_frame_key = j_h2o_frame.key() return H2OFrame.from_java_h2o_frame(j_h2o_frame,j_h2o_frame_key) @staticmethod - def _as_h2o_frame_from_RDD_Int(h2oContext, rdd, framename): - return FrameConversions._as_h2o_frame_from_RDD_Long(h2oContext,rdd, framename) + def _as_h2o_frame_from_RDD_Int(h2oContext, rdd, frame_name): + return FrameConversions._as_h2o_frame_from_RDD_Long(h2oContext,rdd, frame_name) @staticmethod - def _as_h2o_frame_from_RDD_Double(h2oContext, rdd, framename): - j_h2o_frame = h2oContext._jhc.asH2OFrameFromRDDDouble(rdd._to_java_object_rdd(), framename) + def _as_h2o_frame_from_RDD_Double(h2oContext, rdd, frame_name): + j_h2o_frame = h2oContext._jhc.asH2OFrameFromRDDDouble(rdd._to_java_object_rdd(), frame_name) j_h2o_frame_key = j_h2o_frame.key() return H2OFrame.from_java_h2o_frame(j_h2o_frame,j_h2o_frame_key) @staticmethod - def _as_h2o_frame_from_RDD_Float(h2oContext, rdd, framename): - return FrameConversions._as_h2o_frame_from_RDD_Double(h2oContext,rdd, framename) + def _as_h2o_frame_from_RDD_Float(h2oContext, rdd, frame_name): + return FrameConversions._as_h2o_frame_from_RDD_Double(h2oContext,rdd, frame_name) @staticmethod - def _as_h2o_frame_from_RDD_Long(h2oContext, rdd, framename): - j_h2o_frame = h2oContext._jhc.asH2OFrameFromRDDLong(rdd._to_java_object_rdd(), framename) + def _as_h2o_frame_from_RDD_Long(h2oContext, rdd, frame_name): + j_h2o_frame = h2oContext._jhc.asH2OFrameFromRDDLong(rdd._to_java_object_rdd(), frame_name) j_h2o_frame_key = j_h2o_frame.key() return H2OFrame.from_java_h2o_frame(j_h2o_frame,j_h2o_frame_key) @staticmethod - def _as_h2o_frame_from_dataframe(h2oContext, dataframe, framename): + def _as_h2o_frame_from_dataframe(h2oContext, dataframe, frame_name): if dataframe.count() == 0: raise ValueError('Cannot transform empty H2OFrame') - j_h2o_frame = h2oContext._jhc.asH2OFrame(dataframe._jdf, framename) + j_h2o_frame = h2oContext._jhc.asH2OFrame(dataframe._jdf, frame_name) j_h2o_frame_key = j_h2o_frame.key() return H2OFrame.from_java_h2o_frame(j_h2o_frame,j_h2o_frame_key) @staticmethod - def _as_h2o_frame_from_complex_type(h2oContext,dataframe, framename): + def _as_h2o_frame_from_complex_type(h2oContext,dataframe, frame_name): # Creates a DataFrame from an RDD of tuple/list, list or pandas.DataFrame. # On scala backend, to transform RDD of Product to H2OFrame, we need to know Type Tag. # Since there is no alternative for Product class in Python, we first transform the rdd to dataframe # and then transform it to H2OFrame. - df = h2oContext._sqlContext.createDataFrame(dataframe) - j_h2o_frame = h2oContext._jhc.asH2OFrame(df._jdf) + df = h2oContext._sql_context.createDataFrame(dataframe) + j_h2o_frame = h2oContext._jhc.asH2OFrame(df._jdf, frame_name) j_h2o_frame_key = j_h2o_frame.key() return H2OFrame.from_java_h2o_frame(j_h2o_frame,j_h2o_frame_key) diff --git a/py/pysparkling/initializer.py b/py/pysparkling/initializer.py new file mode 100644 index 000000000..2d568a3ad --- /dev/null +++ b/py/pysparkling/initializer.py @@ -0,0 +1,38 @@ +from pkg_resources import resource_filename +import sys +""" +This class is used to load sparkling water JAR into spark environment. It is called when H2OConf or H2OContext is used +to ensure the required java classes are available in Spark. +""" +class Initializer(object): + + # Flag to inform us whether sparkling jar has been already loaded or not. We don't want to load it more than once. + __sparkling_jar_loaded = False + + @staticmethod + def load_sparkling_jar(spark_context): + if not Initializer.__sparkling_jar_loaded: + sys.path.append(".") + Initializer.__add_sparkling_jar_to_spark(spark_context) + Initializer.__sparkling_jar_loaded = True + + @staticmethod + def __add_sparkling_jar_to_spark(sc): + # Add Sparkling water assembly JAR to driver + url = sc._jvm.java.net.URL("file://"+Initializer.__get_sw_jar()) + # Assuming that context class loader is always instance of URLClassLoader ( which should be always true) + cl = sc._jvm.Thread.currentThread().getContextClassLoader() + + # explicitly check if we run on databricks cloud since there we must add the jar to the parent of context class loader + if cl.getClass().getName()=='com.databricks.backend.daemon.driver.DriverLocal$DriverLocalClassLoader': + cl.getParent().getParent().addURL(url) + else: + cl.addURL(url) + + # Add Sparkling water assembly JAR to executors + sc._jsc.addJar(Initializer.__get_sw_jar()) + + @staticmethod + def __get_sw_jar(): + return resource_filename("sparkling_water", 'sparkling_water_assembly.jar') + diff --git a/py/tests/integ_test_utils.py b/py/tests/integ_test_utils.py index cd8acc405..194d2f94a 100644 --- a/py/tests/integ_test_utils.py +++ b/py/tests/integ_test_utils.py @@ -22,35 +22,33 @@ from pyspark import SparkContext, SparkConf import subprocess from random import randrange +import test_utils + class IntegTestEnv: def __init__(self): - self.spark_home = IntegTestEnv.get_env_org_fail("SPARK_HOME","The variable 'SPARK_HOME' should point to Spark home directory.") + self.spark_home = test_utils.get_env_org_fail("SPARK_HOME","The variable 'SPARK_HOME' should point to Spark home directory.") - self.spark_master = IntegTestEnv.get_env_org_fail("MASTER", + self.spark_master = test_utils.get_env_org_fail("MASTER", "The variable 'MASTER' should contain Spark cluster mode.") - self.hdp_version = IntegTestEnv.get_env_org_fail("sparkling.test.hdp.version", + + self.hdp_version = test_utils.get_env_org_fail("sparkling.test.hdp.version", "The variable 'sparkling.test.hdp.version' is not set! It should contain version of hdp used") - self.egg = IntegTestEnv.get_env_org_fail("sparkling.pysparkling.egg", + + self.egg = test_utils.get_env_org_fail("sparkling.pysparkling.egg", "The variable 'sparkling.pysparkling.egg' is not set! It should point to PySparkling egg file") self.spark_conf = {} self.verbose = True - @staticmethod - def get_env_org_fail(prop_name, fail_msg): - try: - return os.environ[prop_name] - except KeyError: - print fail_msg - sys.exit(1) - class IntegTestSuite(unittest.TestCase): + @classmethod def setUpClass(cls): cls.test_env = IntegTestEnv() + @classmethod def tearDownClass(cls): cls.test_env = None diff --git a/py/tests/test_utils.py b/py/tests/test_utils.py index 787088792..a54e9eeca 100644 --- a/py/tests/test_utils.py +++ b/py/tests/test_utils.py @@ -17,10 +17,36 @@ import unittest import os +from pyspark import SparkContext, SparkConf + +def asert_h2o_frame(test_suite, h2o_frame, rdd): + test_suite.assertEquals(h2o_frame.nrow, rdd.count(),"Number of rows should match") + test_suite.assertEquals(h2o_frame.ncol, 1,"Number of columns should equal 1") + test_suite.assertEquals(h2o_frame.names, ["values"],"Column should be name values") + + +def get_default_spark_conf(): + return SparkConf(). \ + setAppName("pyunit-test"). \ + setMaster("local-cluster[3,1,2048]"). \ + set("spark.ext.h2o.disable.ga","true"). \ + set("spark.driver.memory", "2g"). \ + set("spark.executor.memory", "2g"). \ + set("spark.ext.h2o.client.log.level", "DEBUG"). \ + set("spark.ext.h2o.repl.enabled", "false") + +def set_up_class(cls): + pass + + +def tear_down_class(cls): + cls._sc.stop() + # Runs python tests and by default reports to console. # If filename is specified it additionally reports output to file with that name into py/build directory + def run_tests(suite, file_name=None): if file_name is not None: reports_file = 'build'+os.sep+file_name+".txt" @@ -35,4 +61,12 @@ def run_tests(suite, file_name=None): else: # Run tests and print to console testsuite = unittest.TestLoader().loadTestsFromTestCase(suite) - unittest.TextTestRunner(verbosity=2).run(testsuite) \ No newline at end of file + unittest.TextTestRunner(verbosity=2).run(testsuite) + + +def get_env_org_fail(prop_name, fail_msg): + try: + return os.environ[prop_name] + except KeyError: + print fail_msg + sys.exit(1) \ No newline at end of file diff --git a/py/tests/tests_unit.py b/py/tests/tests_unit.py index cc92a9484..34e9f5fb9 100644 --- a/py/tests/tests_unit.py +++ b/py/tests/tests_unit.py @@ -20,32 +20,26 @@ """ import unittest -from pyspark import SparkContext, SparkConf + from pysparkling.context import H2OContext +from pysparkling.conf import H2OConf +from pyspark import SparkContext, SparkConf + import h2o import test_utils -class ReusedPySparklingTestCase(unittest.TestCase): +# Test of transformations from dataframe/rdd to h2o frame and from h2o frame back to dataframe +class FrameTransformationsTest(unittest.TestCase): @classmethod def setUpClass(cls): - conf = SparkConf().setAppName("pyunit-test").setMaster("local-cluster[3,1,2048]").set("spark.ext.h2o.disable.ga","true").set("spark.driver.memory", "2g").set("spark.executor.memory", "2g").set("spark.ext.h2o.client.log.level", "DEBUG") - cls._sc = SparkContext(conf=conf) - cls._hc = H2OContext(cls._sc).start() + cls._sc = SparkContext(conf = test_utils.get_default_spark_conf()) + test_utils.set_up_class(cls) + cls._hc = H2OContext.getOrCreate(cls._sc, H2OConf(cls._sc)) @classmethod def tearDownClass(cls): - cls._sc.stop() - -class TestUtils: - @staticmethod - def asert_h2o_frame(test_suite, h2o_frame, rdd): - test_suite.assertEquals(h2o_frame.nrow, rdd.count(),"Number of rows should match") - test_suite.assertEquals(h2o_frame.ncol, 1,"Number of columns should equal 1") - test_suite.assertEquals(h2o_frame.names, ["values"],"Column should be name values") - -# Test of transformations from dataframe/rdd to h2o frame and from h2o frame back to dataframe -class FrameTransformationsTest(ReusedPySparklingTestCase): + test_utils.tear_down_class(cls) # test transformation from dataframe to h2o frame def test_df_to_h2o_frame(self): @@ -63,7 +57,7 @@ def test_rdd_int_h2o_frame(self): rdd = self._sc.parallelize([num for num in range(0,100)]) h2o_frame = hc.as_h2o_frame(rdd) self.assertEquals(h2o_frame[0,0], 0, "Value should match") - TestUtils.asert_h2o_frame(self,h2o_frame,rdd) + test_utils.asert_h2o_frame(self,h2o_frame,rdd) # test transformation from RDD consisting of python booleans to h2o frame def test_rdd_bool_to_h2o_frame(self): @@ -72,7 +66,7 @@ def test_rdd_bool_to_h2o_frame(self): h2o_frame = hc.as_h2o_frame(rdd) self.assertEquals(h2o_frame[0,0],1,"Value should match") self.assertEquals(h2o_frame[1,0],0,"Value should match") - TestUtils.asert_h2o_frame(self,h2o_frame,rdd) + test_utils.asert_h2o_frame(self,h2o_frame,rdd) # test transformation from RDD consisting of python strings to h2o frame def test_rdd_str_h2o_frame(self): @@ -81,7 +75,7 @@ def test_rdd_str_h2o_frame(self): h2o_frame = hc.as_h2o_frame(rdd) self.assertEquals(h2o_frame[0,0],"a","Value should match") self.assertEquals(h2o_frame[2,0],"c","Value should match") - TestUtils.asert_h2o_frame(self,h2o_frame,rdd) + test_utils.asert_h2o_frame(self,h2o_frame,rdd) # test transformation from RDD consisting of python floats to h2o frame def test_rdd_float_h2o_frame(self): @@ -90,7 +84,7 @@ def test_rdd_float_h2o_frame(self): h2o_frame = hc.as_h2o_frame(rdd) self.assertEquals(h2o_frame[0,0],0.5,"Value should match") self.assertEquals(h2o_frame[1,0],1.3333333333,"Value should match") - TestUtils.asert_h2o_frame(self,h2o_frame,rdd) + test_utils.asert_h2o_frame(self,h2o_frame,rdd) # test transformation from RDD consisting of python complex types to h2o frame def test_rdd_complex_h2o_frame_1(self): @@ -132,7 +126,25 @@ def test_h2o_frame_2_data_frame_2(self): self.assertEquals(len(df.columns), h2o_frame.ncol, "Number of columns should match") self.assertEquals(df.columns,h2o_frame.names, "Column names should match") +class H2OConfTest(unittest.TestCase): + + @classmethod + def setUpClass(cls): + cls._sc = SparkContext(conf = test_utils.get_default_spark_conf().set("spark.ext.h2o.cloud.name", "test-cloud")) + test_utils.set_up_class(cls) + h2o_conf = H2OConf(cls._sc) + cls._hc = H2OContext.getOrCreate(cls._sc, h2o_conf) + + @classmethod + def tearDownClass(cls): + test_utils.tear_down_class(cls) + + # test passing h2o_conf to H2OContext + def test_h2o_conf(self): + self.assertEquals(self._hc.get_conf().cloud_name(),"test-cloud", "Configuration property cloud_name should match") + if __name__ == '__main__': - test_utils.run_tests(FrameTransformationsTest,file_name="py_unit_tests_report") + test_utils.run_tests(FrameTransformationsTest, file_name="py_unit_tests_report") + test_utils.run_tests(H2OConfTest, file_name="py_h2oconf_unit_tests_report") diff --git a/repl/src/main/scala/org/apache/spark/repl/h2o/H2OInterpreter.scala b/repl/src/main/scala/org/apache/spark/repl/h2o/H2OInterpreter.scala index 57ff088e1..80ac16c1a 100644 --- a/repl/src/main/scala/org/apache/spark/repl/h2o/H2OInterpreter.scala +++ b/repl/src/main/scala/org/apache/spark/repl/h2o/H2OInterpreter.scala @@ -39,6 +39,7 @@ import scala.tools.nsc.util._ /** * H2O Interpreter which is use to interpret scala code + * * @param sparkContext spark context * @param sessionId session ID for interpreter */ @@ -65,6 +66,7 @@ class H2OInterpreter(val sparkContext: SparkContext, var sessionId: Int) extends /** * Get response of interpreter + * * @return */ def interpreterResponse: String = { @@ -73,6 +75,7 @@ class H2OInterpreter(val sparkContext: SparkContext, var sessionId: Int) extends /** * Redirected printed output coming from commands written in the interpreter + * * @return */ def consoleOutput: String = { @@ -81,6 +84,7 @@ class H2OInterpreter(val sparkContext: SparkContext, var sessionId: Int) extends /** * Run scala code in a string + * * @param code Code to be compiled end executed * @return */