diff --git a/README.md b/README.md index 16628bd406775..af02339578195 100644 --- a/README.md +++ b/README.md @@ -26,7 +26,7 @@ To build Spark and its example programs, run: (You do not need to do this if you downloaded a pre-built package.) More detailed documentation is available from the project site, at -["Building Spark with Maven"](http://spark.apache.org/docs/latest/building-spark.html). +["Building Spark"](http://spark.apache.org/docs/latest/building-spark.html). ## Interactive Scala Shell diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 9e8d0b785194e..a8c344b1ca594 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -50,8 +50,8 @@ fi if [ -n "$SPARK_PREPEND_CLASSES" ]; then echo "NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark"\ "classes ahead of assembly." >&2 + # Spark classes CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SPARK_SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/core/target/jars/*" CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SPARK_SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SPARK_SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SPARK_SCALA_VERSION/classes" @@ -63,6 +63,8 @@ if [ -n "$SPARK_PREPEND_CLASSES" ]; then CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SPARK_SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/hive-thriftserver/target/scala-$SPARK_SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/yarn/stable/target/scala-$SPARK_SCALA_VERSION/classes" + # Jars for shaded deps in their original form (copied here during build) + CLASSPATH="$CLASSPATH:$FWDIR/core/target/jars/*" fi # Use spark-assembly jar from either RELEASE or assembly directory diff --git a/core/pom.xml b/core/pom.xml index 6fce10a0aea4c..2c115683fce66 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -94,22 +94,45 @@ org.apache.curator curator-recipes + + org.eclipse.jetty jetty-plus + compile org.eclipse.jetty jetty-security + compile org.eclipse.jetty jetty-util + compile org.eclipse.jetty jetty-server + compile + + + org.eclipse.jetty + jetty-http + compile + + org.eclipse.jetty + jetty-continuation + compile + + + org.eclipse.jetty + jetty-servlet + compile + + org.apache.commons commons-lang3 @@ -348,19 +371,24 @@ org.apache.maven.plugins maven-dependency-plugin + copy-dependencies package copy-dependencies - + ${project.build.directory} false false true true - guava + + guava,jetty-io,jetty-servlet,jetty-continuation,jetty-http,jetty-plus,jetty-util,jetty-server + true diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index b28da192c1c0d..5d5288bb6e60d 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -49,6 +49,7 @@ import org.apache.spark.scheduler._ * spark.dynamicAllocation.enabled - Whether this feature is enabled * spark.dynamicAllocation.minExecutors - Lower bound on the number of executors * spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors + * spark.dynamicAllocation.initialExecutors - Number of executors to start with * * spark.dynamicAllocation.schedulerBacklogTimeout (M) - * If there are backlogged tasks for this duration, add new executors @@ -70,9 +71,10 @@ private[spark] class ExecutorAllocationManager( import ExecutorAllocationManager._ - // Lower and upper bounds on the number of executors. These are required. - private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1) - private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1) + // Lower and upper bounds on the number of executors. + private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", 0) + private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", + Integer.MAX_VALUE) // How long there must be backlogged tasks for before an addition is triggered private val schedulerBacklogTimeout = conf.getLong( @@ -132,10 +134,10 @@ private[spark] class ExecutorAllocationManager( */ private def validateSettings(): Unit = { if (minNumExecutors < 0 || maxNumExecutors < 0) { - throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!") + throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be positive!") } - if (minNumExecutors == 0 || maxNumExecutors == 0) { - throw new SparkException("spark.dynamicAllocation.{min/max}Executors cannot be 0!") + if (maxNumExecutors == 0) { + throw new SparkException("spark.dynamicAllocation.maxExecutors cannot be 0!") } if (minNumExecutors > maxNumExecutors) { throw new SparkException(s"spark.dynamicAllocation.minExecutors ($minNumExecutors) must " + diff --git a/core/src/main/scala/org/apache/spark/HttpFileServer.scala b/core/src/main/scala/org/apache/spark/HttpFileServer.scala index 677c5e0f89d72..3f33332a81eaf 100644 --- a/core/src/main/scala/org/apache/spark/HttpFileServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpFileServer.scala @@ -36,7 +36,7 @@ private[spark] class HttpFileServer( var serverUri : String = null def initialize() { - baseDir = Utils.createTempDir() + baseDir = Utils.createTempDir(Utils.getLocalDir(conf), "httpd") fileDir = new File(baseDir, "files") jarDir = new File(baseDir, "jars") fileDir.mkdir() diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala index fa22787ce7ea3..09a9ccc226721 100644 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -19,6 +19,7 @@ package org.apache.spark import java.io.File +import org.eclipse.jetty.server.ssl.SslSocketConnector import org.eclipse.jetty.util.security.{Constraint, Password} import org.eclipse.jetty.security.authentication.DigestAuthenticator import org.eclipse.jetty.security.{ConstraintMapping, ConstraintSecurityHandler, HashLoginService} @@ -72,7 +73,10 @@ private[spark] class HttpServer( */ private def doStart(startPort: Int): (Server, Int) = { val server = new Server() - val connector = new SocketConnector + + val connector = securityManager.fileServerSSLOptions.createJettySslContextFactory() + .map(new SslSocketConnector(_)).getOrElse(new SocketConnector) + connector.setMaxIdleTime(60 * 1000) connector.setSoLingerTime(-1) connector.setPort(startPort) @@ -149,13 +153,14 @@ private[spark] class HttpServer( } /** - * Get the URI of this HTTP server (http://host:port) + * Get the URI of this HTTP server (http://host:port or https://host:port) */ def uri: String = { if (server == null) { throw new ServerStateException("Server is not started") } else { - "http://" + Utils.localIpAddress + ":" + port + val scheme = if (securityManager.fileServerSSLOptions.enabled) "https" else "http" + s"$scheme://${Utils.localIpAddress}:$port" } } } diff --git a/core/src/main/scala/org/apache/spark/SSLOptions.scala b/core/src/main/scala/org/apache/spark/SSLOptions.scala new file mode 100644 index 0000000000000..2cdc167f85af0 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/SSLOptions.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 + +import java.io.File + +import com.typesafe.config.{Config, ConfigFactory, ConfigValueFactory} +import org.eclipse.jetty.util.ssl.SslContextFactory + +/** + * SSLOptions class is a common container for SSL configuration options. It offers methods to + * generate specific objects to configure SSL for different communication protocols. + * + * SSLOptions is intended to provide the maximum common set of SSL settings, which are supported + * by the protocol, which it can generate the configuration for. Since Akka doesn't support client + * authentication with SSL, SSLOptions cannot support it either. + * + * @param enabled enables or disables SSL; if it is set to false, the rest of the + * settings are disregarded + * @param keyStore a path to the key-store file + * @param keyStorePassword a password to access the key-store file + * @param keyPassword a password to access the private key in the key-store + * @param trustStore a path to the trust-store file + * @param trustStorePassword a password to access the trust-store file + * @param protocol SSL protocol (remember that SSLv3 was compromised) supported by Java + * @param enabledAlgorithms a set of encryption algorithms to use + */ +private[spark] case class SSLOptions( + enabled: Boolean = false, + keyStore: Option[File] = None, + keyStorePassword: Option[String] = None, + keyPassword: Option[String] = None, + trustStore: Option[File] = None, + trustStorePassword: Option[String] = None, + protocol: Option[String] = None, + enabledAlgorithms: Set[String] = Set.empty) { + + /** + * Creates a Jetty SSL context factory according to the SSL settings represented by this object. + */ + def createJettySslContextFactory(): Option[SslContextFactory] = { + if (enabled) { + val sslContextFactory = new SslContextFactory() + + keyStore.foreach(file => sslContextFactory.setKeyStorePath(file.getAbsolutePath)) + trustStore.foreach(file => sslContextFactory.setTrustStore(file.getAbsolutePath)) + keyStorePassword.foreach(sslContextFactory.setKeyStorePassword) + trustStorePassword.foreach(sslContextFactory.setTrustStorePassword) + keyPassword.foreach(sslContextFactory.setKeyManagerPassword) + protocol.foreach(sslContextFactory.setProtocol) + sslContextFactory.setIncludeCipherSuites(enabledAlgorithms.toSeq: _*) + + Some(sslContextFactory) + } else { + None + } + } + + /** + * Creates an Akka configuration object which contains all the SSL settings represented by this + * object. It can be used then to compose the ultimate Akka configuration. + */ + def createAkkaConfig: Option[Config] = { + import scala.collection.JavaConversions._ + if (enabled) { + Some(ConfigFactory.empty() + .withValue("akka.remote.netty.tcp.security.key-store", + ConfigValueFactory.fromAnyRef(keyStore.map(_.getAbsolutePath).getOrElse(""))) + .withValue("akka.remote.netty.tcp.security.key-store-password", + ConfigValueFactory.fromAnyRef(keyStorePassword.getOrElse(""))) + .withValue("akka.remote.netty.tcp.security.trust-store", + ConfigValueFactory.fromAnyRef(trustStore.map(_.getAbsolutePath).getOrElse(""))) + .withValue("akka.remote.netty.tcp.security.trust-store-password", + ConfigValueFactory.fromAnyRef(trustStorePassword.getOrElse(""))) + .withValue("akka.remote.netty.tcp.security.key-password", + ConfigValueFactory.fromAnyRef(keyPassword.getOrElse(""))) + .withValue("akka.remote.netty.tcp.security.random-number-generator", + ConfigValueFactory.fromAnyRef("")) + .withValue("akka.remote.netty.tcp.security.protocol", + ConfigValueFactory.fromAnyRef(protocol.getOrElse(""))) + .withValue("akka.remote.netty.tcp.security.enabled-algorithms", + ConfigValueFactory.fromIterable(enabledAlgorithms.toSeq)) + .withValue("akka.remote.netty.tcp.enable-ssl", + ConfigValueFactory.fromAnyRef(true))) + } else { + None + } + } + + /** Returns a string representation of this SSLOptions with all the passwords masked. */ + override def toString: String = s"SSLOptions{enabled=$enabled, " + + s"keyStore=$keyStore, keyStorePassword=${keyStorePassword.map(_ => "xxx")}, " + + s"trustStore=$trustStore, trustStorePassword=${trustStorePassword.map(_ => "xxx")}, " + + s"protocol=$protocol, enabledAlgorithms=$enabledAlgorithms}" + +} + +private[spark] object SSLOptions extends Logging { + + /** Resolves SSLOptions settings from a given Spark configuration object at a given namespace. + * + * The following settings are allowed: + * $ - `[ns].enabled` - `true` or `false`, to enable or disable SSL respectively + * $ - `[ns].keyStore` - a path to the key-store file; can be relative to the current directory + * $ - `[ns].keyStorePassword` - a password to the key-store file + * $ - `[ns].keyPassword` - a password to the private key + * $ - `[ns].trustStore` - a path to the trust-store file; can be relative to the current + * directory + * $ - `[ns].trustStorePassword` - a password to the trust-store file + * $ - `[ns].protocol` - a protocol name supported by a particular Java version + * $ - `[ns].enabledAlgorithms` - a comma separated list of ciphers + * + * For a list of protocols and ciphers supported by particular Java versions, you may go to + * [[https://blogs.oracle.com/java-platform-group/entry/diagnosing_tls_ssl_and_https Oracle + * blog page]]. + * + * You can optionally specify the default configuration. If you do, for each setting which is + * missing in SparkConf, the corresponding setting is used from the default configuration. + * + * @param conf Spark configuration object where the settings are collected from + * @param ns the namespace name + * @param defaults the default configuration + * @return [[org.apache.spark.SSLOptions]] object + */ + def parse(conf: SparkConf, ns: String, defaults: Option[SSLOptions] = None): SSLOptions = { + val enabled = conf.getBoolean(s"$ns.enabled", defaultValue = defaults.exists(_.enabled)) + + val keyStore = conf.getOption(s"$ns.keyStore").map(new File(_)) + .orElse(defaults.flatMap(_.keyStore)) + + val keyStorePassword = conf.getOption(s"$ns.keyStorePassword") + .orElse(defaults.flatMap(_.keyStorePassword)) + + val keyPassword = conf.getOption(s"$ns.keyPassword") + .orElse(defaults.flatMap(_.keyPassword)) + + val trustStore = conf.getOption(s"$ns.trustStore").map(new File(_)) + .orElse(defaults.flatMap(_.trustStore)) + + val trustStorePassword = conf.getOption(s"$ns.trustStorePassword") + .orElse(defaults.flatMap(_.trustStorePassword)) + + val protocol = conf.getOption(s"$ns.protocol") + .orElse(defaults.flatMap(_.protocol)) + + val enabledAlgorithms = conf.getOption(s"$ns.enabledAlgorithms") + .map(_.split(",").map(_.trim).filter(_.nonEmpty).toSet) + .orElse(defaults.map(_.enabledAlgorithms)) + .getOrElse(Set.empty) + + new SSLOptions( + enabled, + keyStore, + keyStorePassword, + keyPassword, + trustStore, + trustStorePassword, + protocol, + enabledAlgorithms) + } + +} + diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index ec82d09cd079b..88d35a4bacc6e 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -18,7 +18,11 @@ package org.apache.spark import java.net.{Authenticator, PasswordAuthentication} +import java.security.KeyStore +import java.security.cert.X509Certificate +import javax.net.ssl._ +import com.google.common.io.Files import org.apache.hadoop.io.Text import org.apache.spark.deploy.SparkHadoopUtil @@ -55,7 +59,7 @@ import org.apache.spark.network.sasl.SecretKeyHolder * Spark also has a set of admin acls (`spark.admin.acls`) which is a set of users/administrators * who always have permission to view or modify the Spark application. * - * Spark does not currently support encryption after authentication. + * Starting from version 1.3, Spark has partial support for encrypted connections with SSL. * * At this point spark has multiple communication protocols that need to be secured and * different underlying mechanisms are used depending on the protocol: @@ -67,8 +71,9 @@ import org.apache.spark.network.sasl.SecretKeyHolder * to connect to the server. There is no control of the underlying * authentication mechanism so its not clear if the password is passed in * plaintext or uses DIGEST-MD5 or some other mechanism. - * Akka also has an option to turn on SSL, this option is not currently supported - * but we could add a configuration option in the future. + * + * Akka also has an option to turn on SSL, this option is currently supported (see + * the details below). * * - HTTP for broadcast and file server (via HttpServer) -> Spark currently uses Jetty * for the HttpServer. Jetty supports multiple authentication mechanisms - @@ -77,8 +82,9 @@ import org.apache.spark.network.sasl.SecretKeyHolder * to authenticate using DIGEST-MD5 via a single user and the shared secret. * Since we are using DIGEST-MD5, the shared secret is not passed on the wire * in plaintext. - * We currently do not support SSL (https), but Jetty can be configured to use it - * so we could add a configuration option for this in the future. + * + * We currently support SSL (https) for this communication protocol (see the details + * below). * * The Spark HttpServer installs the HashLoginServer and configures it to DIGEST-MD5. * Any clients must specify the user and password. There is a default @@ -142,9 +148,40 @@ import org.apache.spark.network.sasl.SecretKeyHolder * authentication. Spark will then use that user to compare against the view acls to do * authorization. If not filter is in place the user is generally null and no authorization * can take place. + * + * Connection encryption (SSL) configuration is organized hierarchically. The user can configure + * the default SSL settings which will be used for all the supported communication protocols unless + * they are overwritten by protocol specific settings. This way the user can easily provide the + * common settings for all the protocols without disabling the ability to configure each one + * individually. + * + * All the SSL settings like `spark.ssl.xxx` where `xxx` is a particular configuration property, + * denote the global configuration for all the supported protocols. In order to override the global + * configuration for the particular protocol, the properties must be overwritten in the + * protocol-specific namespace. Use `spark.ssl.yyy.xxx` settings to overwrite the global + * configuration for particular protocol denoted by `yyy`. Currently `yyy` can be either `akka` for + * Akka based connections or `fs` for broadcast and file server. + * + * Refer to [[org.apache.spark.SSLOptions]] documentation for the list of + * options that can be specified. + * + * SecurityManager initializes SSLOptions objects for different protocols separately. SSLOptions + * object parses Spark configuration at a given namespace and builds the common representation + * of SSL settings. SSLOptions is then used to provide protocol-specific configuration like + * TypeSafe configuration for Akka or SSLContextFactory for Jetty. + * + * SSL must be configured on each node and configured for each component involved in + * communication using the particular protocol. In YARN clusters, the key-store can be prepared on + * the client side then distributed and used by the executors as the part of the application + * (YARN allows the user to deploy files before the application is started). + * In standalone deployment, the user needs to provide key-stores and configuration + * options for master and workers. In this mode, the user may allow the executors to use the SSL + * settings inherited from the worker which spawned that executor. It can be accomplished by + * setting `spark.ssl.useNodeLocalConf` to `true`. */ -private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging with SecretKeyHolder { +private[spark] class SecurityManager(sparkConf: SparkConf) + extends Logging with SecretKeyHolder { // key used to store the spark secret in the Hadoop UGI private val sparkSecretLookupKey = "sparkCookie" @@ -196,6 +233,57 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging with ) } + // the default SSL configuration - it will be used by all communication layers unless overwritten + private val defaultSSLOptions = SSLOptions.parse(sparkConf, "spark.ssl", defaults = None) + + // SSL configuration for different communication layers - they can override the default + // configuration at a specified namespace. The namespace *must* start with spark.ssl. + val fileServerSSLOptions = SSLOptions.parse(sparkConf, "spark.ssl.fs", Some(defaultSSLOptions)) + val akkaSSLOptions = SSLOptions.parse(sparkConf, "spark.ssl.akka", Some(defaultSSLOptions)) + + logDebug(s"SSLConfiguration for file server: $fileServerSSLOptions") + logDebug(s"SSLConfiguration for Akka: $akkaSSLOptions") + + val (sslSocketFactory, hostnameVerifier) = if (fileServerSSLOptions.enabled) { + val trustStoreManagers = + for (trustStore <- fileServerSSLOptions.trustStore) yield { + val input = Files.asByteSource(fileServerSSLOptions.trustStore.get).openStream() + + try { + val ks = KeyStore.getInstance(KeyStore.getDefaultType) + ks.load(input, fileServerSSLOptions.trustStorePassword.get.toCharArray) + + val tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm) + tmf.init(ks) + tmf.getTrustManagers + } finally { + input.close() + } + } + + lazy val credulousTrustStoreManagers = Array({ + logWarning("Using 'accept-all' trust manager for SSL connections.") + new X509TrustManager { + override def getAcceptedIssuers: Array[X509Certificate] = null + + override def checkClientTrusted(x509Certificates: Array[X509Certificate], s: String) {} + + override def checkServerTrusted(x509Certificates: Array[X509Certificate], s: String) {} + }: TrustManager + }) + + val sslContext = SSLContext.getInstance(fileServerSSLOptions.protocol.getOrElse("Default")) + sslContext.init(null, trustStoreManagers.getOrElse(credulousTrustStoreManagers), null) + + val hostVerifier = new HostnameVerifier { + override def verify(s: String, sslSession: SSLSession): Boolean = true + } + + (Some(sslContext.getSocketFactory), Some(hostVerifier)) + } else { + (None, None) + } + /** * Split a comma separated String, filter out any empty items, and return a Set of strings */ diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index cd91c8f87547b..13aa9960ac33a 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -23,6 +23,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.LinkedHashSet import org.apache.spark.serializer.KryoSerializer +import org.apache.spark.util.Utils /** * Configuration for a Spark application. Used to set various Spark parameters as key-value pairs. @@ -53,8 +54,8 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { if (loadDefaults) { // Load any spark.* system properties - for ((k, v) <- System.getProperties.asScala if k.startsWith("spark.")) { - set(k, v) + for ((key, value) <- Utils.getSystemProperties if key.startsWith("spark.")) { + set(key, value) } } @@ -369,6 +370,7 @@ private[spark] object SparkConf { isAkkaConf(name) || name.startsWith("spark.akka") || name.startsWith("spark.auth") || + name.startsWith("spark.ssl") || isSparkPortConf(name) } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3c61c10820ba9..228076f01c841 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -687,9 +687,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * @param minPartitions Minimum number of Hadoop Splits to generate. * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each - * record, directly caching the returned RDD will create many references to the same object. - * If you plan to directly cache Hadoop writable objects, you should first copy them using - * a `map` function. + * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle + * operation will create many references to the same object. + * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first + * copy them using a `map` function. */ def hadoopRDD[K, V]( conf: JobConf, @@ -705,12 +706,13 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } /** Get an RDD for a Hadoop file with an arbitrary InputFormat - * - * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each - * record, directly caching the returned RDD will create many references to the same object. - * If you plan to directly cache Hadoop writable objects, you should first copy them using - * a `map` function. - * */ + * + * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each + * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle + * operation will create many references to the same object. + * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first + * copy them using a `map` function. + */ def hadoopFile[K, V]( path: String, inputFormatClass: Class[_ <: InputFormat[K, V]], @@ -741,9 +743,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * }}} * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each - * record, directly caching the returned RDD will create many references to the same object. - * If you plan to directly cache Hadoop writable objects, you should first copy them using - * a `map` function. + * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle + * operation will create many references to the same object. + * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first + * copy them using a `map` function. */ def hadoopFile[K, V, F <: InputFormat[K, V]] (path: String, minPartitions: Int) @@ -764,9 +767,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * }}} * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each - * record, directly caching the returned RDD will create many references to the same object. - * If you plan to directly cache Hadoop writable objects, you should first copy them using - * a `map` function. + * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle + * operation will create many references to the same object. + * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first + * copy them using a `map` function. */ def hadoopFile[K, V, F <: InputFormat[K, V]](path: String) (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = @@ -788,9 +792,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * and extra configuration options to pass to the input format. * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each - * record, directly caching the returned RDD will create many references to the same object. - * If you plan to directly cache Hadoop writable objects, you should first copy them using - * a `map` function. + * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle + * operation will create many references to the same object. + * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first + * copy them using a `map` function. */ def newAPIHadoopFile[K, V, F <: NewInputFormat[K, V]]( path: String, @@ -810,9 +815,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * and extra configuration options to pass to the input format. * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each - * record, directly caching the returned RDD will create many references to the same object. - * If you plan to directly cache Hadoop writable objects, you should first copy them using - * a `map` function. + * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle + * operation will create many references to the same object. + * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first + * copy them using a `map` function. */ def newAPIHadoopRDD[K, V, F <: NewInputFormat[K, V]]( conf: Configuration = hadoopConfiguration, @@ -826,9 +832,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli /** Get an RDD for a Hadoop SequenceFile with given key and value types. * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each - * record, directly caching the returned RDD will create many references to the same object. - * If you plan to directly cache Hadoop writable objects, you should first copy them using - * a `map` function. + * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle + * operation will create many references to the same object. + * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first + * copy them using a `map` function. */ def sequenceFile[K, V](path: String, keyClass: Class[K], @@ -843,9 +850,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli /** Get an RDD for a Hadoop SequenceFile with given key and value types. * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each - * record, directly caching the returned RDD will create many references to the same object. - * If you plan to directly cache Hadoop writable objects, you should first copy them using - * a `map` function. + * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle + * operation will create many references to the same object. + * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first + * copy them using a `map` function. * */ def sequenceFile[K, V](path: String, keyClass: Class[K], valueClass: Class[V]): RDD[(K, V)] = { assertNotStopped() @@ -869,9 +877,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * allow it to figure out the Writable class to use in the subclass case. * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each - * record, directly caching the returned RDD will create many references to the same object. - * If you plan to directly cache Hadoop writable objects, you should first copy them using - * a `map` function. + * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle + * operation will create many references to the same object. + * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first + * copy them using a `map` function. */ def sequenceFile[K, V] (path: String, minPartitions: Int = defaultMinPartitions) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 1264a8126153b..f25db7f8de565 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -339,7 +339,7 @@ object SparkEnv extends Logging { // this is a temporary directory; in distributed mode, this is the executor's current working // directory. val sparkFilesDir: String = if (isDriver) { - Utils.createTempDir().getAbsolutePath + Utils.createTempDir(Utils.getLocalDir(conf), "userFiles").getAbsolutePath } else { "." } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 119e0459c5d1b..b89effc16d36d 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -316,6 +316,7 @@ private object SpecialLengths { val PYTHON_EXCEPTION_THROWN = -2 val TIMING_DATA = -3 val END_OF_STREAM = -4 + val NULL = -5 } private[spark] object PythonRDD extends Logging { @@ -374,54 +375,25 @@ private[spark] object PythonRDD extends Logging { } def writeIteratorToStream[T](iter: Iterator[T], dataOut: DataOutputStream) { - // The right way to implement this would be to use TypeTags to get the full - // type of T. Since I don't want to introduce breaking changes throughout the - // entire Spark API, I have to use this hacky approach: - if (iter.hasNext) { - val first = iter.next() - val newIter = Seq(first).iterator ++ iter - first match { - case arr: Array[Byte] => - newIter.asInstanceOf[Iterator[Array[Byte]]].foreach { bytes => - dataOut.writeInt(bytes.length) - dataOut.write(bytes) - } - case string: String => - newIter.asInstanceOf[Iterator[String]].foreach { str => - writeUTF(str, dataOut) - } - case stream: PortableDataStream => - newIter.asInstanceOf[Iterator[PortableDataStream]].foreach { stream => - val bytes = stream.toArray() - dataOut.writeInt(bytes.length) - dataOut.write(bytes) - } - case (key: String, stream: PortableDataStream) => - newIter.asInstanceOf[Iterator[(String, PortableDataStream)]].foreach { - case (key, stream) => - writeUTF(key, dataOut) - val bytes = stream.toArray() - dataOut.writeInt(bytes.length) - dataOut.write(bytes) - } - case (key: String, value: String) => - newIter.asInstanceOf[Iterator[(String, String)]].foreach { - case (key, value) => - writeUTF(key, dataOut) - writeUTF(value, dataOut) - } - case (key: Array[Byte], value: Array[Byte]) => - newIter.asInstanceOf[Iterator[(Array[Byte], Array[Byte])]].foreach { - case (key, value) => - dataOut.writeInt(key.length) - dataOut.write(key) - dataOut.writeInt(value.length) - dataOut.write(value) - } - case other => - throw new SparkException("Unexpected element type " + first.getClass) - } + + def write(obj: Any): Unit = obj match { + case null => + dataOut.writeInt(SpecialLengths.NULL) + case arr: Array[Byte] => + dataOut.writeInt(arr.length) + dataOut.write(arr) + case str: String => + writeUTF(str, dataOut) + case stream: PortableDataStream => + write(stream.toArray()) + case (key, value) => + write(key) + write(value) + case other => + throw new SparkException("Unexpected element type " + other.getClass) } + + iter.foreach(write) } /** diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index be5ebfa9219d3..b7cfc8bd9c542 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -22,6 +22,7 @@ import java.io.{File, InputStream, IOException, OutputStream} import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkContext +import org.apache.spark.api.java.{JavaSparkContext, JavaRDD} private[spark] object PythonUtils { /** Get the PYTHONPATH for PySpark, either from SPARK_HOME, if it is set, or from our JAR */ @@ -39,4 +40,8 @@ private[spark] object PythonUtils { def mergePythonPaths(paths: String*): String = { paths.filter(_ != "").mkString(File.pathSeparator) } + + def generateRDDWithNull(sc: JavaSparkContext): JavaRDD[String] = { + sc.parallelize(List("a", null, "b")) + } } diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index 31d6958c403b3..1444c0dd3d2d6 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -151,7 +151,7 @@ private[broadcast] object HttpBroadcast extends Logging { } private def createServer(conf: SparkConf) { - broadcastDir = Utils.createTempDir(Utils.getLocalDir(conf)) + broadcastDir = Utils.createTempDir(Utils.getLocalDir(conf), "broadcast") val broadcastPort = conf.getInt("spark.broadcast.port", 0) server = new HttpServer(conf, broadcastDir, securityManager, broadcastPort, "HTTP broadcast server") @@ -199,6 +199,7 @@ private[broadcast] object HttpBroadcast extends Logging { uc = new URL(url).openConnection() uc.setConnectTimeout(httpReadTimeout) } + Utils.setupSecureURLConnection(uc, securityManager) val in = { uc.setReadTimeout(httpReadTimeout) diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index 65a1a8fd7e929..ae55b4ff40b74 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -28,5 +28,14 @@ private[spark] class ApplicationDescription( val user = System.getProperty("user.name", "") + def copy( + name: String = name, + maxCores: Option[Int] = maxCores, + memoryPerSlave: Int = memoryPerSlave, + command: Command = command, + appUiUrl: String = appUiUrl, + eventLogDir: Option[String] = eventLogDir): ApplicationDescription = + new ApplicationDescription(name, maxCores, memoryPerSlave, command, appUiUrl, eventLogDir) + override def toString: String = "ApplicationDescription(" + name + ")" } diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index 7c1c831c248fc..38b3da0b13756 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -39,7 +39,8 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) val timeout = AkkaUtils.askTimeout(conf) override def preStart() = { - masterActor = context.actorSelection(Master.toAkkaUrl(driverArgs.master)) + masterActor = context.actorSelection( + Master.toAkkaUrl(driverArgs.master, AkkaUtils.protocol(context.system))) context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) @@ -161,7 +162,7 @@ object Client { "driverClient", Utils.localHostName(), 0, conf, new SecurityManager(conf)) // Verify driverArgs.master is a valid url so that we can use it in ClientActor safely - Master.toAkkaUrl(driverArgs.master) + Master.toAkkaUrl(driverArgs.master, AkkaUtils.protocol(actorSystem)) actorSystem.actorOf(Props(classOf[ClientActor], driverArgs, conf)) actorSystem.awaitTermination() diff --git a/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala b/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala index 58c95dc4f9116..b056a19ce6598 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala @@ -25,5 +25,13 @@ private[spark] class DriverDescription( val command: Command) extends Serializable { + def copy( + jarUrl: String = jarUrl, + mem: Int = mem, + cores: Int = cores, + supervise: Boolean = supervise, + command: Command = command): DriverDescription = + new DriverDescription(jarUrl, mem, cores, supervise, command) + override def toString: String = s"DriverDescription (${command.mainClass})" } diff --git a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala index 039c8719e2867..53e18c4bcec23 100644 --- a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala @@ -26,7 +26,7 @@ import org.apache.spark.api.python.PythonUtils import org.apache.spark.util.{RedirectThread, Utils} /** - * A main class used by spark-submit to launch Python applications. It executes python as a + * A main class used to launch Python applications. It executes python as a * subprocess and then has it connect back to the JVM to access system properties, etc. */ object PythonRunner { diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 211e3ede53d9c..d68854214ef06 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -141,7 +141,7 @@ class SparkHadoopUtil extends Logging { val baselineBytesRead = f() Some(() => f() - baselineBytesRead) } catch { - case e: NoSuchMethodException => { + case e @ (_: NoSuchMethodException | _: ClassNotFoundException) => { logDebug("Couldn't find method for retrieving thread-level FileSystem input data", e) None } @@ -163,7 +163,7 @@ class SparkHadoopUtil extends Logging { val baselineBytesWritten = f() Some(() => f() - baselineBytesWritten) } catch { - case e: NoSuchMethodException => { + case e @ (_: NoSuchMethodException | _: ClassNotFoundException) => { logDebug("Couldn't find method for retrieving thread-level FileSystem output data", e) None } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index c240bcd705d93..02021be9f93d4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -23,6 +23,8 @@ import java.net.URL import scala.collection.mutable.{ArrayBuffer, HashMap, Map} +import org.apache.hadoop.fs.Path + import org.apache.spark.executor.ExecutorURLClassLoader import org.apache.spark.util.Utils @@ -134,12 +136,27 @@ object SparkSubmit { } } + val isYarnCluster = clusterManager == YARN && deployMode == CLUSTER + + // Require all python files to be local, so we can add them to the PYTHONPATH + // In YARN cluster mode, python files are distributed as regular files, which can be non-local + if (args.isPython && !isYarnCluster) { + if (Utils.nonLocalPaths(args.primaryResource).nonEmpty) { + printErrorAndExit(s"Only local python files are supported: $args.primaryResource") + } + val nonLocalPyFiles = Utils.nonLocalPaths(args.pyFiles).mkString(",") + if (nonLocalPyFiles.nonEmpty) { + printErrorAndExit(s"Only local additional python files are supported: $nonLocalPyFiles") + } + } + // The following modes are not supported or applicable (clusterManager, deployMode) match { case (MESOS, CLUSTER) => printErrorAndExit("Cluster deploy mode is currently not supported for Mesos clusters.") - case (_, CLUSTER) if args.isPython => - printErrorAndExit("Cluster deploy mode is currently not supported for python applications.") + case (STANDALONE, CLUSTER) if args.isPython => + printErrorAndExit("Cluster deploy mode is currently not supported for python " + + "applications on standalone clusters.") case (_, CLUSTER) if isShell(args.primaryResource) => printErrorAndExit("Cluster deploy mode is not applicable to Spark shells.") case (_, CLUSTER) if isSqlShell(args.mainClass) => @@ -150,7 +167,7 @@ object SparkSubmit { } // If we're running a python app, set the main class to our specific python runner - if (args.isPython) { + if (args.isPython && deployMode == CLIENT) { if (args.primaryResource == PYSPARK_SHELL) { args.mainClass = "py4j.GatewayServer" args.childArgs = ArrayBuffer("--die-on-broken-pipe", "0") @@ -167,6 +184,13 @@ object SparkSubmit { } } + // In yarn-cluster mode for a python app, add primary resource and pyFiles to files + // that can be distributed with the job + if (args.isPython && isYarnCluster) { + args.files = mergeFileLists(args.files, args.primaryResource) + args.files = mergeFileLists(args.files, args.pyFiles) + } + // Special flag to avoid deprecation warnings at the client sysProps("SPARK_SUBMIT") = "true" @@ -245,7 +269,6 @@ object SparkSubmit { // Add the application jar automatically so the user doesn't have to call sc.addJar // For YARN cluster mode, the jar is already distributed on each node as "app.jar" // For python files, the primary resource is already distributed as a regular file - val isYarnCluster = clusterManager == YARN && deployMode == CLUSTER if (!isYarnCluster && !args.isPython) { var jars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq.empty) if (isUserJar(args.primaryResource)) { @@ -270,10 +293,22 @@ object SparkSubmit { // In yarn-cluster mode, use yarn.Client as a wrapper around the user class if (isYarnCluster) { childMainClass = "org.apache.spark.deploy.yarn.Client" - if (args.primaryResource != SPARK_INTERNAL) { - childArgs += ("--jar", args.primaryResource) + if (args.isPython) { + val mainPyFile = new Path(args.primaryResource).getName + childArgs += ("--primary-py-file", mainPyFile) + if (args.pyFiles != null) { + // These files will be distributed to each machine's working directory, so strip the + // path prefix + val pyFilesNames = args.pyFiles.split(",").map(p => (new Path(p)).getName).mkString(",") + childArgs += ("--py-files", pyFilesNames) + } + childArgs += ("--class", "org.apache.spark.deploy.PythonRunner") + } else { + if (args.primaryResource != SPARK_INTERNAL) { + childArgs += ("--jar", args.primaryResource) + } + childArgs += ("--class", args.mainClass) } - childArgs += ("--class", args.mainClass) if (args.childArgs != null) { args.childArgs.foreach { arg => childArgs += ("--arg", arg) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 81ec08cb6d501..73e921fd83ef2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -179,18 +179,6 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St SparkSubmit.printErrorAndExit("--py-files given but primary resource is not a Python script") } - // Require all python files to be local, so we can add them to the PYTHONPATH - if (isPython) { - if (Utils.nonLocalPaths(primaryResource).nonEmpty) { - SparkSubmit.printErrorAndExit(s"Only local python files are supported: $primaryResource") - } - val nonLocalPyFiles = Utils.nonLocalPaths(pyFiles).mkString(",") - if (nonLocalPyFiles.nonEmpty) { - SparkSubmit.printErrorAndExit( - s"Only local additional python files are supported: $nonLocalPyFiles") - } - } - if (master.startsWith("yarn")) { val hasHadoopEnv = env.contains("HADOOP_CONF_DIR") || env.contains("YARN_CONF_DIR") if (!hasHadoopEnv && !Utils.isTesting) { diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index 39a7b0319b6a1..ffe940fbda2fb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -47,7 +47,7 @@ private[spark] class AppClient( conf: SparkConf) extends Logging { - val masterAkkaUrls = masterUrls.map(Master.toAkkaUrl) + val masterAkkaUrls = masterUrls.map(Master.toAkkaUrl(_, AkkaUtils.protocol(actorSystem))) val REGISTRATION_TIMEOUT = 20.seconds val REGISTRATION_RETRIES = 3 @@ -107,8 +107,9 @@ private[spark] class AppClient( def changeMaster(url: String) { // activeMasterUrl is a valid Spark url since we receive it from master. activeMasterUrl = url - master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl)) - masterAddress = Master.toAkkaAddress(activeMasterUrl) + master = context.actorSelection( + Master.toAkkaUrl(activeMasterUrl, AkkaUtils.protocol(actorSystem))) + masterAddress = Master.toAkkaAddress(activeMasterUrl, AkkaUtils.protocol(actorSystem)) } private def isPossibleMaster(remoteUrl: Address) = { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index d92d99310a583..5eeb9fe526248 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -860,9 +860,9 @@ private[spark] object Master extends Logging { * * @throws SparkException if the url is invalid */ - def toAkkaUrl(sparkUrl: String): String = { + def toAkkaUrl(sparkUrl: String, protocol: String): String = { val (host, port) = Utils.extractHostPortFromSparkUrl(sparkUrl) - "akka.tcp://%s@%s:%s/user/%s".format(systemName, host, port, actorName) + AkkaUtils.address(protocol, systemName, host, port, actorName) } /** @@ -870,9 +870,9 @@ private[spark] object Master extends Logging { * * @throws SparkException if the url is invalid */ - def toAkkaAddress(sparkUrl: String): Address = { + def toAkkaAddress(sparkUrl: String, protocol: String): Address = { val (host, port) = Utils.extractHostPortFromSparkUrl(sparkUrl) - Address("akka.tcp", systemName, host, port) + Address(protocol, systemName, host, port) } def startSystemAndActor( diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index acbdf0d8bd7bc..bc9f78b9e5c77 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -26,7 +26,7 @@ import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files import org.apache.spark.{SparkConf, Logging} -import org.apache.spark.deploy.{ApplicationDescription, Command, ExecutorState} +import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged import org.apache.spark.util.logging.FileAppender diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 13599830123d0..b20f5c0c82895 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -31,8 +31,8 @@ import scala.util.Random import akka.actor._ import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} -import org.apache.spark.deploy.{ExecutorDescription, ExecutorState} +import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.deploy.{Command, ExecutorDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.{DriverState, Master} import org.apache.spark.deploy.worker.ui.WorkerWebUI @@ -93,7 +93,12 @@ private[spark] class Worker( var masterAddress: Address = null var activeMasterUrl: String = "" var activeMasterWebUiUrl : String = "" - val akkaUrl = "akka.tcp://%s@%s:%s/user/%s".format(actorSystemName, host, port, actorName) + val akkaUrl = AkkaUtils.address( + AkkaUtils.protocol(context.system), + actorSystemName, + host, + port, + actorName) @volatile var registered = false @volatile var connected = false val workerId = generateWorkerId() @@ -174,8 +179,9 @@ private[spark] class Worker( // activeMasterUrl it's a valid Spark url since we receive it from master. activeMasterUrl = url activeMasterWebUiUrl = uiUrl - master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl)) - masterAddress = Master.toAkkaAddress(activeMasterUrl) + master = context.actorSelection( + Master.toAkkaUrl(activeMasterUrl, AkkaUtils.protocol(context.system))) + masterAddress = Master.toAkkaAddress(activeMasterUrl, AkkaUtils.protocol(context.system)) connected = true // Cancel any outstanding re-registration attempts because we found a new master registrationRetryTimer.foreach(_.cancel()) @@ -347,10 +353,20 @@ private[spark] class Worker( }.toSeq } appDirectories(appId) = appLocalDirs - - val manager = new ExecutorRunner(appId, execId, appDesc, cores_, memory_, - self, workerId, host, sparkHome, executorDir, akkaUrl, conf, appLocalDirs, - ExecutorState.LOADING) + val manager = new ExecutorRunner( + appId, + execId, + appDesc.copy(command = Worker.maybeUpdateSSLSettings(appDesc.command, conf)), + cores_, + memory_, + self, + workerId, + host, + sparkHome, + executorDir, + akkaUrl, + conf, + appLocalDirs, ExecutorState.LOADING) executors(appId + "/" + execId) = manager manager.start() coresUsed += cores_ @@ -406,7 +422,14 @@ private[spark] class Worker( case LaunchDriver(driverId, driverDesc) => { logInfo(s"Asked to launch driver $driverId") - val driver = new DriverRunner(conf, driverId, workDir, sparkHome, driverDesc, self, akkaUrl) + val driver = new DriverRunner( + conf, + driverId, + workDir, + sparkHome, + driverDesc.copy(command = Worker.maybeUpdateSSLSettings(driverDesc.command, conf)), + self, + akkaUrl) drivers(driverId) = driver driver.start() @@ -523,10 +546,32 @@ private[spark] object Worker extends Logging { val securityMgr = new SecurityManager(conf) val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, conf = conf, securityManager = securityMgr) - val masterAkkaUrls = masterUrls.map(Master.toAkkaUrl) + val masterAkkaUrls = masterUrls.map(Master.toAkkaUrl(_, AkkaUtils.protocol(actorSystem))) actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory, masterAkkaUrls, systemName, actorName, workDir, conf, securityMgr), name = actorName) (actorSystem, boundPort) } + private[spark] def isUseLocalNodeSSLConfig(cmd: Command): Boolean = { + val pattern = """\-Dspark\.ssl\.useNodeLocalConf\=(.+)""".r + val result = cmd.javaOpts.collectFirst { + case pattern(_result) => _result.toBoolean + } + result.getOrElse(false) + } + + private[spark] def maybeUpdateSSLSettings(cmd: Command, conf: SparkConf): Command = { + val prefix = "spark.ssl." + val useNLC = "spark.ssl.useNodeLocalConf" + if (isUseLocalNodeSSLConfig(cmd)) { + val newJavaOpts = cmd.javaOpts + .filter(opt => !opt.startsWith(s"-D$prefix")) ++ + conf.getAll.collect { case (key, value) if key.startsWith(prefix) => s"-D$key=$value" } :+ + s"-D$useNLC=true" + cmd.copy(javaOpts = newJavaOpts) + } else { + cmd + } + } + } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 823825302658c..bc72c8970319c 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -123,7 +123,11 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { val executorConf = new SparkConf val port = executorConf.getInt("spark.executor.port", 0) val (fetcher, _) = AkkaUtils.createActorSystem( - "driverPropsFetcher", hostname, port, executorConf, new SecurityManager(executorConf)) + "driverPropsFetcher", + hostname, + port, + executorConf, + new SecurityManager(executorConf)) val driver = fetcher.actorSelection(driverUrl) val timeout = AkkaUtils.askTimeout(executorConf) val fut = Patterns.ask(driver, RetrieveSparkProps, timeout) @@ -132,7 +136,15 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { fetcher.shutdown() // Create SparkEnv using properties we fetched from the driver. - val driverConf = new SparkConf().setAll(props) + val driverConf = new SparkConf() + for ((key, value) <- props) { + // this is required for SSL in standalone mode + if (SparkConf.isExecutorStartupConf(key)) { + driverConf.setIfMissing(key, value) + } else { + driverConf.set(key, value) + } + } val env = SparkEnv.createExecutorEnv( driverConf, executorId, hostname, port, cores, isLocal = false) diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index c3e3931042de2..89adddcf0ac36 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -42,10 +42,11 @@ import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.executor.{DataReadMethod, InputMetrics} +import org.apache.spark.executor.DataReadMethod import org.apache.spark.rdd.HadoopRDD.HadoopMapPartitionsWithSplitRDD import org.apache.spark.util.{NextIterator, Utils} import org.apache.spark.scheduler.{HostTaskLocation, HDFSCacheTaskLocation} +import org.apache.spark.storage.StorageLevel /** * A Spark split class that wraps around a Hadoop InputSplit. @@ -308,6 +309,15 @@ class HadoopRDD[K, V]( // Do nothing. Hadoop RDD should not be checkpointed. } + override def persist(storageLevel: StorageLevel): this.type = { + if (storageLevel.deserialized) { + logWarning("Caching NewHadoopRDDs as deserialized objects usually leads to undesired" + + " behavior because Hadoop's RecordReader reuses the same Writable object for all records." + + " Use a map transformation to make copies of the records.") + } + super.persist(storageLevel) + } + def getConf: Configuration = getJobConf() } diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index d86f95ac3e485..44b9ffd2a53fd 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -29,16 +29,13 @@ import org.apache.hadoop.mapreduce.lib.input.{CombineFileSplit, FileSplit} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.input.WholeTextFileInputFormat -import org.apache.spark.InterruptibleIterator -import org.apache.spark.Logging -import org.apache.spark.Partition -import org.apache.spark.SerializableWritable -import org.apache.spark.{SparkContext, TaskContext} +import org.apache.spark._ import org.apache.spark.executor.DataReadMethod import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD import org.apache.spark.util.Utils import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.storage.StorageLevel private[spark] class NewHadoopPartition( rddId: Int, @@ -211,6 +208,16 @@ class NewHadoopRDD[K, V]( locs.getOrElse(split.getLocations.filter(_ != "localhost")) } + override def persist(storageLevel: StorageLevel): this.type = { + if (storageLevel.deserialized) { + logWarning("Caching NewHadoopRDDs as deserialized objects usually leads to undesired" + + " behavior because Hadoop's RecordReader reuses the same Writable object for all records." + + " Use a map transformation to make copies of the records.") + } + super.persist(storageLevel) + } + + def getConf: Configuration = confBroadcast.value.value } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala index ee10aa061f4e9..06786a59524e7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.{Path, FileSystem} import org.apache.spark.{Logging, SparkContext, SparkEnv} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.scheduler.TaskSchedulerImpl +import org.apache.spark.util.AkkaUtils private[spark] class SimrSchedulerBackend( scheduler: TaskSchedulerImpl, @@ -38,7 +39,8 @@ private[spark] class SimrSchedulerBackend( override def start() { super.start() - val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + val driverUrl = AkkaUtils.address( + AkkaUtils.protocol(actorSystem), SparkEnv.driverActorSystemName, sc.conf.get("spark.driver.host"), sc.conf.get("spark.driver.port"), diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 7eb87a564d6f5..d2e1680a5fd1b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -21,7 +21,7 @@ import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv} import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.client.{AppClient, AppClientListener} import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SlaveLost, TaskSchedulerImpl} -import org.apache.spark.util.Utils +import org.apache.spark.util.{AkkaUtils, Utils} private[spark] class SparkDeploySchedulerBackend( scheduler: TaskSchedulerImpl, @@ -46,7 +46,8 @@ private[spark] class SparkDeploySchedulerBackend( super.start() // The endpoint for executors to talk to us - val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + val driverUrl = AkkaUtils.address( + AkkaUtils.protocol(actorSystem), SparkEnv.driverActorSystemName, conf.get("spark.driver.host"), conf.get("spark.driver.port"), diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 5289661eb896b..0d1c2a916ca7f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -31,7 +31,7 @@ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTas import org.apache.spark.{Logging, SparkContext, SparkEnv, SparkException} import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend -import org.apache.spark.util.Utils +import org.apache.spark.util.{Utils, AkkaUtils} /** * A SchedulerBackend that runs tasks on Mesos, but uses "coarse-grained" tasks, where it holds @@ -143,7 +143,8 @@ private[spark] class CoarseMesosSchedulerBackend( } val command = CommandInfo.newBuilder() .setEnvironment(environment) - val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + val driverUrl = AkkaUtils.address( + AkkaUtils.protocol(sc.env.actorSystem), SparkEnv.driverActorSystemName, conf.get("spark.driver.host"), conf.get("spark.driver.port"), diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index af05eb3ca69ce..53eaedacbf291 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -17,9 +17,8 @@ package org.apache.spark.storage +import java.util.UUID import java.io.{IOException, File} -import java.text.SimpleDateFormat -import java.util.{Date, Random, UUID} import org.apache.spark.{SparkConf, Logging} import org.apache.spark.executor.ExecutorExitCode @@ -37,7 +36,6 @@ import org.apache.spark.util.Utils private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkConf) extends Logging { - private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 private[spark] val subDirsPerLocalDir = blockManager.conf.getInt("spark.diskStore.subDirectories", 64) @@ -123,33 +121,15 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon } private def createLocalDirs(conf: SparkConf): Array[File] = { - val dateFormat = new SimpleDateFormat("yyyyMMddHHmmss") Utils.getOrCreateLocalRootDirs(conf).flatMap { rootDir => - var foundLocalDir = false - var localDir: File = null - var localDirId: String = null - var tries = 0 - val rand = new Random() - while (!foundLocalDir && tries < MAX_DIR_CREATION_ATTEMPTS) { - tries += 1 - try { - localDirId = "%s-%04x".format(dateFormat.format(new Date), rand.nextInt(65536)) - localDir = new File(rootDir, s"spark-local-$localDirId") - if (!localDir.exists) { - foundLocalDir = localDir.mkdirs() - } - } catch { - case e: Exception => - logWarning(s"Attempt $tries to create local dir $localDir failed", e) - } - } - if (!foundLocalDir) { - logError(s"Failed $MAX_DIR_CREATION_ATTEMPTS attempts to create local dir in $rootDir." + - " Ignoring this directory.") - None - } else { + try { + val localDir = Utils.createDirectory(rootDir, "blockmgr") logInfo(s"Created local directory at $localDir") Some(localDir) + } catch { + case e: IOException => + logError(s"Failed to create local dir in $rootDir. Ignoring this directory.", e) + None } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala index 479f967fb1541..527f960af2dfc 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala @@ -128,7 +128,7 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { activeStagesTable.toNodeSeq } if (shouldShowPendingStages) { - content ++=

Pending Stages ({pendingStages.size}

++ + content ++=

Pending Stages ({pendingStages.size})

++ pendingStagesTable.toNodeSeq } if (shouldShowCompletedStages) { diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 4c9b1e3c46f0f..3d9c6192ff7f7 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -20,6 +20,7 @@ package org.apache.spark.util import scala.collection.JavaConversions.mapAsJavaMap import scala.concurrent.Await import scala.concurrent.duration.{Duration, FiniteDuration} +import scala.util.Try import akka.actor.{ActorRef, ActorSystem, ExtendedActorSystem} import akka.pattern.ask @@ -91,8 +92,11 @@ private[spark] object AkkaUtils extends Logging { val secureCookie = if (isAuthOn) secretKey else "" logDebug(s"In createActorSystem, requireCookie is: $requireCookie") - val akkaConf = ConfigFactory.parseMap(conf.getAkkaConf.toMap[String, String]).withFallback( - ConfigFactory.parseString( + val akkaSslConfig = securityManager.akkaSSLOptions.createAkkaConfig + .getOrElse(ConfigFactory.empty()) + + val akkaConf = ConfigFactory.parseMap(conf.getAkkaConf.toMap[String, String]) + .withFallback(akkaSslConfig).withFallback(ConfigFactory.parseString( s""" |akka.daemonic = on |akka.loggers = [""akka.event.slf4j.Slf4jLogger""] @@ -214,7 +218,7 @@ private[spark] object AkkaUtils extends Logging { val driverHost: String = conf.get("spark.driver.host", "localhost") val driverPort: Int = conf.getInt("spark.driver.port", 7077) Utils.checkHost(driverHost, "Expected hostname") - val url = s"akka.tcp://$driverActorSystemName@$driverHost:$driverPort/user/$name" + val url = address(protocol(actorSystem), driverActorSystemName, driverHost, driverPort, name) val timeout = AkkaUtils.lookupTimeout(conf) logInfo(s"Connecting to $name: $url") Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout) @@ -228,9 +232,33 @@ private[spark] object AkkaUtils extends Logging { actorSystem: ActorSystem): ActorRef = { val executorActorSystemName = SparkEnv.executorActorSystemName Utils.checkHost(host, "Expected hostname") - val url = s"akka.tcp://$executorActorSystemName@$host:$port/user/$name" + val url = address(protocol(actorSystem), executorActorSystemName, host, port, name) val timeout = AkkaUtils.lookupTimeout(conf) logInfo(s"Connecting to $name: $url") Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout) } + + def protocol(actorSystem: ActorSystem): String = { + val akkaConf = actorSystem.settings.config + val sslProp = "akka.remote.netty.tcp.enable-ssl" + protocol(akkaConf.hasPath(sslProp) && akkaConf.getBoolean(sslProp)) + } + + def protocol(ssl: Boolean = false): String = { + if (ssl) { + "akka.ssl.tcp" + } else { + "akka.tcp" + } + } + + def address( + protocol: String, + systemName: String, + host: String, + port: Any, + actorName: String): String = { + s"$protocol://$systemName@$host:$port/user/$actorName" + } + } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 414bc49a57f8a..8e0e41ad3782e 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -32,7 +32,6 @@ import org.apache.spark.executor._ import org.apache.spark.scheduler._ import org.apache.spark.storage._ import org.apache.spark._ -import org.apache.hadoop.hdfs.web.JsonUtil /** * Serializes SparkListener events to/from JSON. This protocol provides strong backwards- diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 86ac307fc84ba..e9f2aed9ffbea 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -21,8 +21,9 @@ import java.io._ import java.lang.management.ManagementFactory import java.net._ import java.nio.ByteBuffer -import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadFactory, ThreadPoolExecutor} -import java.util.{Locale, Properties, Random, UUID} +import java.util.{Properties, Locale, Random, UUID} +import java.util.concurrent.{ThreadFactory, ConcurrentHashMap, Executors, ThreadPoolExecutor} +import javax.net.ssl.HttpsURLConnection import scala.collection.JavaConversions._ import scala.collection.Map @@ -60,6 +61,8 @@ private[spark] object CallSite { private[spark] object Utils extends Logging { val random = new Random() + private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 + /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream() @@ -246,13 +249,28 @@ private[spark] object Utils extends Logging { retval } + /** + * JDK equivalent of `chmod 700 file`. + * + * @param file the file whose permissions will be modified + * @return true if the permissions were successfully changed, false otherwise. + */ + def chmod700(file: File): Boolean = { + file.setReadable(false, false) && + file.setReadable(true, true) && + file.setWritable(false, false) && + file.setWritable(true, true) && + file.setExecutable(false, false) && + file.setExecutable(true, true) + } + /** * Create a directory inside the given parent directory. The directory is guaranteed to be * newly created, and is not marked for automatic deletion. */ - def createDirectory(root: String): File = { + def createDirectory(root: String, namePrefix: String = "spark"): File = { var attempts = 0 - val maxAttempts = 10 + val maxAttempts = MAX_DIR_CREATION_ATTEMPTS var dir: File = null while (dir == null) { attempts += 1 @@ -264,6 +282,13 @@ private[spark] object Utils extends Logging { dir = new File(root, "spark-" + UUID.randomUUID.toString) if (dir.exists() || !dir.mkdirs()) { dir = null + } else { + // Restrict file permissions via chmod if available. + // For Windows this step is ignored. + if (!isWindows && !chmod700(dir)) { + dir.delete() + dir = null + } } } catch { case e: SecurityException => dir = null; } } @@ -275,8 +300,10 @@ private[spark] object Utils extends Logging { * Create a temporary directory inside the given parent directory. The directory will be * automatically deleted when the VM shuts down. */ - def createTempDir(root: String = System.getProperty("java.io.tmpdir")): File = { - val dir = createDirectory(root) + def createTempDir( + root: String = System.getProperty("java.io.tmpdir"), + namePrefix: String = "spark"): File = { + val dir = createDirectory(root, namePrefix) registerShutdownDeleteDir(dir) dir } @@ -549,6 +576,7 @@ private[spark] object Utils extends Logging { logDebug("fetchFile not using security") uc = new URL(url).openConnection() } + Utils.setupSecureURLConnection(uc, securityMgr) val timeout = conf.getInt("spark.files.fetchTimeout", 60) * 1000 uc.setConnectTimeout(timeout) @@ -599,26 +627,35 @@ private[spark] object Utils extends Logging { * If no directories could be created, this will return an empty list. */ private[spark] def getOrCreateLocalRootDirs(conf: SparkConf): Array[String] = { - val confValue = if (isRunningInYarnContainer(conf)) { + if (isRunningInYarnContainer(conf)) { // If we are in yarn mode, systems can have different disk layouts so we must set it - // to what Yarn on this system said was available. - getYarnLocalDirs(conf) + // to what Yarn on this system said was available. Note this assumes that Yarn has + // created the directories already, and that they are secured so that only the + // user has access to them. + getYarnLocalDirs(conf).split(",") } else { - Option(conf.getenv("SPARK_LOCAL_DIRS")).getOrElse( - conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))) - } - val rootDirs = confValue.split(',') - logDebug(s"Getting/creating local root dirs at '$confValue'") - - rootDirs.flatMap { rootDir => - val localDir: File = new File(rootDir) - val foundLocalDir = localDir.exists || localDir.mkdirs() - if (!foundLocalDir) { - logError(s"Failed to create local root dir in $rootDir. Ignoring this directory.") - None - } else { - Some(rootDir) - } + // In non-Yarn mode (or for the driver in yarn-client mode), we cannot trust the user + // configuration to point to a secure directory. So create a subdirectory with restricted + // permissions under each listed directory. + Option(conf.getenv("SPARK_LOCAL_DIRS")) + .getOrElse(conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))) + .split(",") + .flatMap { root => + try { + val rootDir = new File(root) + if (rootDir.exists || rootDir.mkdirs()) { + Some(createDirectory(root).getAbsolutePath()) + } else { + logError(s"Failed to create dir in $root. Ignoring this directory.") + None + } + } catch { + case e: IOException => + logError(s"Failed to create local root dir in $root. Ignoring this directory.") + None + } + } + .toArray } } @@ -1312,9 +1349,14 @@ private[spark] object Utils extends Logging { hashAbs } - /** Returns a copy of the system properties that is thread-safe to iterator over. */ - def getSystemProperties(): Map[String, String] = { - System.getProperties.clone().asInstanceOf[java.util.Properties].toMap[String, String] + /** Returns the system properties map that is thread-safe to iterator over. It gets the + * properties which have been set explicitly, as well as those for which only a default value + * has been defined. */ + def getSystemProperties: Map[String, String] = { + val sysProps = for (key <- System.getProperties.stringPropertyNames()) yield + (key, System.getProperty(key)) + + sysProps.toMap } /** @@ -1780,6 +1822,20 @@ private[spark] object Utils extends Logging { PropertyConfigurator.configure(pro) } + /** + * If the given URL connection is HttpsURLConnection, it sets the SSL socket factory and + * the host verifier from the given security manager. + */ + def setupSecureURLConnection(urlConnection: URLConnection, sm: SecurityManager): URLConnection = { + urlConnection match { + case https: HttpsURLConnection => + sm.sslSocketFactory.foreach(https.setSSLSocketFactory) + sm.hostnameVerifier.foreach(https.setHostnameVerifier) + https + case connection => connection + } + } + def invoke( clazz: Class[_], obj: AnyRef, diff --git a/core/src/test/resources/keystore b/core/src/test/resources/keystore new file mode 100644 index 0000000000000..f8310e39ba1e0 Binary files /dev/null and b/core/src/test/resources/keystore differ diff --git a/core/src/test/resources/truststore b/core/src/test/resources/truststore new file mode 100644 index 0000000000000..a6b1d46e1f391 Binary files /dev/null and b/core/src/test/resources/truststore differ diff --git a/core/src/test/resources/untrusted-keystore b/core/src/test/resources/untrusted-keystore new file mode 100644 index 0000000000000..6015b02caa128 Binary files /dev/null and b/core/src/test/resources/untrusted-keystore differ diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 0e4df17c1bf87..57081ddd959a5 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -32,24 +32,23 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { import ExecutorAllocationManagerSuite._ test("verify min/max executors") { - // No min or max val conf = new SparkConf() .setMaster("local") .setAppName("test-executor-allocation-manager") .set("spark.dynamicAllocation.enabled", "true") .set("spark.dynamicAllocation.testing", "true") - intercept[SparkException] { new SparkContext(conf) } - SparkEnv.get.stop() // cleanup the created environment - SparkContext.clearActiveContext() + val sc0 = new SparkContext(conf) + assert(sc0.executorAllocationManager.isDefined) + sc0.stop() - // Only min - val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "1") + // Min < 0 + val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "-1") intercept[SparkException] { new SparkContext(conf1) } SparkEnv.get.stop() SparkContext.clearActiveContext() - // Only max - val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "2") + // Max < 0 + val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "-1") intercept[SparkException] { new SparkContext(conf2) } SparkEnv.get.stop() SparkContext.clearActiveContext() diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index 0f49ce4754fbb..5fdf6bc2777e3 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -18,13 +18,19 @@ package org.apache.spark import java.io._ +import java.net.URI import java.util.jar.{JarEntry, JarOutputStream} +import javax.net.ssl.SSLHandshakeException import com.google.common.io.ByteStreams +import org.apache.commons.io.{FileUtils, IOUtils} +import org.apache.commons.lang3.RandomUtils import org.scalatest.FunSuite import org.apache.spark.util.Utils +import SSLSampleConfigs._ + class FileServerSuite extends FunSuite with LocalSparkContext { @transient var tmpDir: File = _ @@ -168,4 +174,88 @@ class FileServerSuite extends FunSuite with LocalSparkContext { } } + test ("HttpFileServer should work with SSL") { + val sparkConf = sparkSSLConfig() + val sm = new SecurityManager(sparkConf) + val server = new HttpFileServer(sparkConf, sm, 0) + try { + server.initialize() + + fileTransferTest(server, sm) + } finally { + server.stop() + } + } + + test ("HttpFileServer should work with SSL and good credentials") { + val sparkConf = sparkSSLConfig() + sparkConf.set("spark.authenticate", "true") + sparkConf.set("spark.authenticate.secret", "good") + + val sm = new SecurityManager(sparkConf) + val server = new HttpFileServer(sparkConf, sm, 0) + try { + server.initialize() + + fileTransferTest(server, sm) + } finally { + server.stop() + } + } + + test ("HttpFileServer should not work with valid SSL and bad credentials") { + val sparkConf = sparkSSLConfig() + sparkConf.set("spark.authenticate", "true") + sparkConf.set("spark.authenticate.secret", "bad") + + val sm = new SecurityManager(sparkConf) + val server = new HttpFileServer(sparkConf, sm, 0) + try { + server.initialize() + + intercept[IOException] { + fileTransferTest(server) + } + } finally { + server.stop() + } + } + + test ("HttpFileServer should not work with SSL when the server is untrusted") { + val sparkConf = sparkSSLConfigUntrusted() + val sm = new SecurityManager(sparkConf) + val server = new HttpFileServer(sparkConf, sm, 0) + try { + server.initialize() + + intercept[SSLHandshakeException] { + fileTransferTest(server) + } + } finally { + server.stop() + } + } + + def fileTransferTest(server: HttpFileServer, sm: SecurityManager = null): Unit = { + val randomContent = RandomUtils.nextBytes(100) + val file = File.createTempFile("FileServerSuite", "sslTests", tmpDir) + FileUtils.writeByteArrayToFile(file, randomContent) + server.addFile(file) + + val uri = new URI(server.serverUri + "/files/" + file.getName) + + val connection = if (sm != null && sm.isAuthenticationEnabled()) { + Utils.constructURIForAuthentication(uri, sm).toURL.openConnection() + } else { + uri.toURL.openConnection() + } + + if (sm != null) { + Utils.setupSecureURLConnection(connection, sm) + } + + val buf = IOUtils.toByteArray(connection.getInputStream) + assert(buf === randomContent) + } + } diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index d27880f4bc32f..ccfe0678cb1c3 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -120,7 +120,7 @@ class MapOutputTrackerSuite extends FunSuite { securityManager = new SecurityManager(conf)) val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( - s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") + AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) val timeout = AkkaUtils.lookupTimeout(conf) slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) diff --git a/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala b/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala new file mode 100644 index 0000000000000..444a33371bd71 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala @@ -0,0 +1,123 @@ +/* + * 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 + +import java.io.File + +import com.google.common.io.Files +import org.apache.spark.util.Utils +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +class SSLOptionsSuite extends FunSuite with BeforeAndAfterAll { + + test("test resolving property file as spark conf ") { + val keyStorePath = new File(this.getClass.getResource("/keystore").toURI).getAbsolutePath + val trustStorePath = new File(this.getClass.getResource("/truststore").toURI).getAbsolutePath + + val conf = new SparkConf + conf.set("spark.ssl.enabled", "true") + conf.set("spark.ssl.keyStore", keyStorePath) + conf.set("spark.ssl.keyStorePassword", "password") + conf.set("spark.ssl.keyPassword", "password") + conf.set("spark.ssl.trustStore", trustStorePath) + conf.set("spark.ssl.trustStorePassword", "password") + conf.set("spark.ssl.enabledAlgorithms", "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") + conf.set("spark.ssl.protocol", "SSLv3") + + val opts = SSLOptions.parse(conf, "spark.ssl") + + assert(opts.enabled === true) + assert(opts.trustStore.isDefined === true) + assert(opts.trustStore.get.getName === "truststore") + assert(opts.trustStore.get.getAbsolutePath === trustStorePath) + assert(opts.keyStore.isDefined === true) + assert(opts.keyStore.get.getName === "keystore") + assert(opts.keyStore.get.getAbsolutePath === keyStorePath) + assert(opts.trustStorePassword === Some("password")) + assert(opts.keyStorePassword === Some("password")) + assert(opts.keyPassword === Some("password")) + assert(opts.protocol === Some("SSLv3")) + assert(opts.enabledAlgorithms === Set("TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA")) + } + + test("test resolving property with defaults specified ") { + val keyStorePath = new File(this.getClass.getResource("/keystore").toURI).getAbsolutePath + val trustStorePath = new File(this.getClass.getResource("/truststore").toURI).getAbsolutePath + + val conf = new SparkConf + conf.set("spark.ssl.enabled", "true") + conf.set("spark.ssl.keyStore", keyStorePath) + conf.set("spark.ssl.keyStorePassword", "password") + conf.set("spark.ssl.keyPassword", "password") + conf.set("spark.ssl.trustStore", trustStorePath) + conf.set("spark.ssl.trustStorePassword", "password") + conf.set("spark.ssl.enabledAlgorithms", "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") + conf.set("spark.ssl.protocol", "SSLv3") + + val defaultOpts = SSLOptions.parse(conf, "spark.ssl", defaults = None) + val opts = SSLOptions.parse(conf, "spark.ui.ssl", defaults = Some(defaultOpts)) + + assert(opts.enabled === true) + assert(opts.trustStore.isDefined === true) + assert(opts.trustStore.get.getName === "truststore") + assert(opts.trustStore.get.getAbsolutePath === trustStorePath) + assert(opts.keyStore.isDefined === true) + assert(opts.keyStore.get.getName === "keystore") + assert(opts.keyStore.get.getAbsolutePath === keyStorePath) + assert(opts.trustStorePassword === Some("password")) + assert(opts.keyStorePassword === Some("password")) + assert(opts.keyPassword === Some("password")) + assert(opts.protocol === Some("SSLv3")) + assert(opts.enabledAlgorithms === Set("TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA")) + } + + test("test whether defaults can be overridden ") { + val keyStorePath = new File(this.getClass.getResource("/keystore").toURI).getAbsolutePath + val trustStorePath = new File(this.getClass.getResource("/truststore").toURI).getAbsolutePath + + val conf = new SparkConf + conf.set("spark.ssl.enabled", "true") + conf.set("spark.ui.ssl.enabled", "false") + conf.set("spark.ssl.keyStore", keyStorePath) + conf.set("spark.ssl.keyStorePassword", "password") + conf.set("spark.ui.ssl.keyStorePassword", "12345") + conf.set("spark.ssl.keyPassword", "password") + conf.set("spark.ssl.trustStore", trustStorePath) + conf.set("spark.ssl.trustStorePassword", "password") + conf.set("spark.ssl.enabledAlgorithms", "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") + conf.set("spark.ui.ssl.enabledAlgorithms", "ABC, DEF") + conf.set("spark.ssl.protocol", "SSLv3") + + val defaultOpts = SSLOptions.parse(conf, "spark.ssl", defaults = None) + val opts = SSLOptions.parse(conf, "spark.ui.ssl", defaults = Some(defaultOpts)) + + assert(opts.enabled === false) + assert(opts.trustStore.isDefined === true) + assert(opts.trustStore.get.getName === "truststore") + assert(opts.trustStore.get.getAbsolutePath === trustStorePath) + assert(opts.keyStore.isDefined === true) + assert(opts.keyStore.get.getName === "keystore") + assert(opts.keyStore.get.getAbsolutePath === keyStorePath) + assert(opts.trustStorePassword === Some("password")) + assert(opts.keyStorePassword === Some("12345")) + assert(opts.keyPassword === Some("password")) + assert(opts.protocol === Some("SSLv3")) + assert(opts.enabledAlgorithms === Set("ABC", "DEF")) + } + +} diff --git a/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala b/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala new file mode 100644 index 0000000000000..ace8123a8961f --- /dev/null +++ b/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala @@ -0,0 +1,55 @@ +/* + * 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 + +import java.io.File + +object SSLSampleConfigs { + val keyStorePath = new File(this.getClass.getResource("/keystore").toURI).getAbsolutePath + val untrustedKeyStorePath = new File(this.getClass.getResource("/untrusted-keystore").toURI).getAbsolutePath + val trustStorePath = new File(this.getClass.getResource("/truststore").toURI).getAbsolutePath + + def sparkSSLConfig() = { + val conf = new SparkConf(loadDefaults = false) + conf.set("spark.ssl.enabled", "true") + conf.set("spark.ssl.keyStore", keyStorePath) + conf.set("spark.ssl.keyStorePassword", "password") + conf.set("spark.ssl.keyPassword", "password") + conf.set("spark.ssl.trustStore", trustStorePath) + conf.set("spark.ssl.trustStorePassword", "password") + conf.set("spark.ssl.enabledAlgorithms", + "TLS_RSA_WITH_AES_128_CBC_SHA, SSL_RSA_WITH_DES_CBC_SHA") + conf.set("spark.ssl.protocol", "TLSv1") + conf + } + + def sparkSSLConfigUntrusted() = { + val conf = new SparkConf(loadDefaults = false) + conf.set("spark.ssl.enabled", "true") + conf.set("spark.ssl.keyStore", untrustedKeyStorePath) + conf.set("spark.ssl.keyStorePassword", "password") + conf.set("spark.ssl.keyPassword", "password") + conf.set("spark.ssl.trustStore", trustStorePath) + conf.set("spark.ssl.trustStorePassword", "password") + conf.set("spark.ssl.enabledAlgorithms", + "TLS_RSA_WITH_AES_128_CBC_SHA, SSL_RSA_WITH_DES_CBC_SHA") + conf.set("spark.ssl.protocol", "TLSv1") + conf + } + +} diff --git a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala index fcca0867b8072..43fbd3ff3f756 100644 --- a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark -import scala.collection.mutable.ArrayBuffer +import java.io.File import org.scalatest.FunSuite @@ -125,6 +125,54 @@ class SecurityManagerSuite extends FunSuite { } + test("ssl on setup") { + val conf = SSLSampleConfigs.sparkSSLConfig() + + val securityManager = new SecurityManager(conf) + + assert(securityManager.fileServerSSLOptions.enabled === true) + assert(securityManager.akkaSSLOptions.enabled === true) + + assert(securityManager.sslSocketFactory.isDefined === true) + assert(securityManager.hostnameVerifier.isDefined === true) + + assert(securityManager.fileServerSSLOptions.trustStore.isDefined === true) + assert(securityManager.fileServerSSLOptions.trustStore.get.getName === "truststore") + assert(securityManager.fileServerSSLOptions.keyStore.isDefined === true) + assert(securityManager.fileServerSSLOptions.keyStore.get.getName === "keystore") + assert(securityManager.fileServerSSLOptions.trustStorePassword === Some("password")) + assert(securityManager.fileServerSSLOptions.keyStorePassword === Some("password")) + assert(securityManager.fileServerSSLOptions.keyPassword === Some("password")) + assert(securityManager.fileServerSSLOptions.protocol === Some("TLSv1")) + assert(securityManager.fileServerSSLOptions.enabledAlgorithms === + Set("TLS_RSA_WITH_AES_128_CBC_SHA", "SSL_RSA_WITH_DES_CBC_SHA")) + + assert(securityManager.akkaSSLOptions.trustStore.isDefined === true) + assert(securityManager.akkaSSLOptions.trustStore.get.getName === "truststore") + assert(securityManager.akkaSSLOptions.keyStore.isDefined === true) + assert(securityManager.akkaSSLOptions.keyStore.get.getName === "keystore") + assert(securityManager.akkaSSLOptions.trustStorePassword === Some("password")) + assert(securityManager.akkaSSLOptions.keyStorePassword === Some("password")) + assert(securityManager.akkaSSLOptions.keyPassword === Some("password")) + assert(securityManager.akkaSSLOptions.protocol === Some("TLSv1")) + assert(securityManager.akkaSSLOptions.enabledAlgorithms === + Set("TLS_RSA_WITH_AES_128_CBC_SHA", "SSL_RSA_WITH_DES_CBC_SHA")) + } + + test("ssl off setup") { + val file = File.createTempFile("SSLOptionsSuite", "conf") + file.deleteOnExit() + + System.setProperty("spark.ssl.configFile", file.getAbsolutePath) + val conf = new SparkConf() + + val securityManager = new SecurityManager(conf) + + assert(securityManager.fileServerSSLOptions.enabled === false) + assert(securityManager.akkaSSLOptions.enabled === false) + assert(securityManager.sslSocketFactory.isDefined === false) + assert(securityManager.hostnameVerifier.isDefined === false) + } } diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 790976a5ac308..e08210ae60d17 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -17,6 +17,10 @@ package org.apache.spark +import java.util.concurrent.{TimeUnit, Executors} + +import scala.util.{Try, Random} + import org.scalatest.FunSuite import org.apache.spark.serializer.{KryoRegistrator, KryoSerializer} import org.apache.spark.util.ResetSystemProperties @@ -123,6 +127,27 @@ class SparkConfSuite extends FunSuite with LocalSparkContext with ResetSystemPro assert(conf.get("spark.test.a.b.c") === "a.b.c") } + test("Thread safeness - SPARK-5425") { + import scala.collection.JavaConversions._ + val executor = Executors.newSingleThreadScheduledExecutor() + val sf = executor.scheduleAtFixedRate(new Runnable { + override def run(): Unit = + System.setProperty("spark.5425." + Random.nextInt(), Random.nextInt().toString) + }, 0, 1, TimeUnit.MILLISECONDS) + + try { + val t0 = System.currentTimeMillis() + while ((System.currentTimeMillis() - t0) < 1000) { + val conf = Try(new SparkConf(loadDefaults = true)) + assert(conf.isSuccess === true) + } + } finally { + executor.shutdownNow() + for (key <- System.getProperties.stringPropertyNames() if key.startsWith("spark.5425.")) + System.getProperties.remove(key) + } + } + test("register kryo classes through registerKryoClasses") { val conf = new SparkConf().set("spark.kryo.registrationRequired", "true") diff --git a/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala b/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala index 7b866f08a0e9f..c63d834f9048b 100644 --- a/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala @@ -23,11 +23,22 @@ import org.scalatest.FunSuite class PythonRDDSuite extends FunSuite { - test("Writing large strings to the worker") { - val input: List[String] = List("a"*100000) - val buffer = new DataOutputStream(new ByteArrayOutputStream) - PythonRDD.writeIteratorToStream(input.iterator, buffer) - } + test("Writing large strings to the worker") { + val input: List[String] = List("a"*100000) + val buffer = new DataOutputStream(new ByteArrayOutputStream) + PythonRDD.writeIteratorToStream(input.iterator, buffer) + } + test("Handle nulls gracefully") { + val buffer = new DataOutputStream(new ByteArrayOutputStream) + // Should not have NPE when write an Iterator with null in it + // The correctness will be tested in Python + PythonRDD.writeIteratorToStream(Iterator("a", null), buffer) + PythonRDD.writeIteratorToStream(Iterator(null, "a"), buffer) + PythonRDD.writeIteratorToStream(Iterator("a".getBytes, null), buffer) + PythonRDD.writeIteratorToStream(Iterator(null, "a".getBytes), buffer) + PythonRDD.writeIteratorToStream(Iterator((null, null), ("a", null), (null, "b")), buffer) + PythonRDD.writeIteratorToStream( + Iterator((null, null), ("a".getBytes, null), (null, "b".getBytes)), buffer) + } } - diff --git a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala index d2dae34be7bfb..518073dcbb64e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala @@ -23,6 +23,7 @@ import org.scalatest.Matchers class ClientSuite extends FunSuite with Matchers { test("correctly validates driver jar URL's") { ClientArguments.isValidJarUrl("http://someHost:8080/foo.jar") should be (true) + ClientArguments.isValidJarUrl("https://someHost:8080/foo.jar") should be (true) // file scheme with authority and path is valid. ClientArguments.isValidJarUrl("file://somehost/path/to/a/jarFile.jar") should be (true) diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index 3d2335f9b3637..34c74d87f0a62 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -20,30 +20,46 @@ package org.apache.spark.deploy.master import akka.actor.Address import org.scalatest.FunSuite -import org.apache.spark.SparkException +import org.apache.spark.{SSLOptions, SparkConf, SparkException} class MasterSuite extends FunSuite { test("toAkkaUrl") { - val akkaUrl = Master.toAkkaUrl("spark://1.2.3.4:1234") + val conf = new SparkConf(loadDefaults = false) + val akkaUrl = Master.toAkkaUrl("spark://1.2.3.4:1234", "akka.tcp") assert("akka.tcp://sparkMaster@1.2.3.4:1234/user/Master" === akkaUrl) } + test("toAkkaUrl with SSL") { + val conf = new SparkConf(loadDefaults = false) + val akkaUrl = Master.toAkkaUrl("spark://1.2.3.4:1234", "akka.ssl.tcp") + assert("akka.ssl.tcp://sparkMaster@1.2.3.4:1234/user/Master" === akkaUrl) + } + test("toAkkaUrl: a typo url") { + val conf = new SparkConf(loadDefaults = false) val e = intercept[SparkException] { - Master.toAkkaUrl("spark://1.2. 3.4:1234") + Master.toAkkaUrl("spark://1.2. 3.4:1234", "akka.tcp") } assert("Invalid master URL: spark://1.2. 3.4:1234" === e.getMessage) } test("toAkkaAddress") { - val address = Master.toAkkaAddress("spark://1.2.3.4:1234") + val conf = new SparkConf(loadDefaults = false) + val address = Master.toAkkaAddress("spark://1.2.3.4:1234", "akka.tcp") assert(Address("akka.tcp", "sparkMaster", "1.2.3.4", 1234) === address) } + test("toAkkaAddress with SSL") { + val conf = new SparkConf(loadDefaults = false) + val address = Master.toAkkaAddress("spark://1.2.3.4:1234", "akka.ssl.tcp") + assert(Address("akka.ssl.tcp", "sparkMaster", "1.2.3.4", 1234) === address) + } + test("toAkkaAddress: a typo url") { + val conf = new SparkConf(loadDefaults = false) val e = intercept[SparkException] { - Master.toAkkaAddress("spark://1.2. 3.4:1234") + Master.toAkkaAddress("spark://1.2. 3.4:1234", "akka.tcp") } assert("Invalid master URL: spark://1.2. 3.4:1234" === e.getMessage) } diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala new file mode 100644 index 0000000000000..84e2fd7ad936d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala @@ -0,0 +1,57 @@ +/* + * 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.deploy.worker + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.Command + +import org.scalatest.{Matchers, FunSuite} + +class WorkerSuite extends FunSuite with Matchers { + + def cmd(javaOpts: String*) = Command("", Seq.empty, Map.empty, Seq.empty, Seq.empty, Seq(javaOpts:_*)) + def conf(opts: (String, String)*) = new SparkConf(loadDefaults = false).setAll(opts) + + test("test isUseLocalNodeSSLConfig") { + Worker.isUseLocalNodeSSLConfig(cmd("-Dasdf=dfgh")) shouldBe false + Worker.isUseLocalNodeSSLConfig(cmd("-Dspark.ssl.useNodeLocalConf=true")) shouldBe true + Worker.isUseLocalNodeSSLConfig(cmd("-Dspark.ssl.useNodeLocalConf=false")) shouldBe false + Worker.isUseLocalNodeSSLConfig(cmd("-Dspark.ssl.useNodeLocalConf=")) shouldBe false + } + + test("test maybeUpdateSSLSettings") { + Worker.maybeUpdateSSLSettings( + cmd("-Dasdf=dfgh", "-Dspark.ssl.opt1=x"), + conf("spark.ssl.opt1" -> "y", "spark.ssl.opt2" -> "z")) + .javaOpts should contain theSameElementsInOrderAs Seq( + "-Dasdf=dfgh", "-Dspark.ssl.opt1=x") + + Worker.maybeUpdateSSLSettings( + cmd("-Dspark.ssl.useNodeLocalConf=false", "-Dspark.ssl.opt1=x"), + conf("spark.ssl.opt1" -> "y", "spark.ssl.opt2" -> "z")) + .javaOpts should contain theSameElementsInOrderAs Seq( + "-Dspark.ssl.useNodeLocalConf=false", "-Dspark.ssl.opt1=x") + + Worker.maybeUpdateSSLSettings( + cmd("-Dspark.ssl.useNodeLocalConf=true", "-Dspark.ssl.opt1=x"), + conf("spark.ssl.opt1" -> "y", "spark.ssl.opt2" -> "z")) + .javaOpts should contain theSameElementsAs Seq( + "-Dspark.ssl.useNodeLocalConf=true", "-Dspark.ssl.opt1=y", "-Dspark.ssl.opt2=z") + + } +} diff --git a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala index 6bbf72e929dcb..39e5d367d676c 100644 --- a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.util +import java.util.concurrent.TimeoutException + import scala.concurrent.Await import akka.actor._ @@ -26,6 +28,7 @@ import org.scalatest.FunSuite import org.apache.spark._ import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage.BlockManagerId +import org.apache.spark.SSLSampleConfigs._ /** @@ -47,7 +50,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") val badconf = new SparkConf badconf.set("spark.authenticate", "true") @@ -60,7 +63,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro conf = conf, securityManager = securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( - s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") + AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) val timeout = AkkaUtils.lookupTimeout(conf) intercept[akka.actor.ActorNotFound] { slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) @@ -74,7 +77,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val conf = new SparkConf conf.set("spark.authenticate", "false") conf.set("spark.authenticate.secret", "bad") - val securityManager = new SecurityManager(conf); + val securityManager = new SecurityManager(conf) val hostname = "localhost" val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, @@ -85,18 +88,18 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") val badconf = new SparkConf badconf.set("spark.authenticate", "false") badconf.set("spark.authenticate.secret", "good") - val securityManagerBad = new SecurityManager(badconf); + val securityManagerBad = new SecurityManager(badconf) val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = badconf, securityManager = securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( - s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") + AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) val timeout = AkkaUtils.lookupTimeout(conf) slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) @@ -124,7 +127,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val conf = new SparkConf conf.set("spark.authenticate", "true") conf.set("spark.authenticate.secret", "good") - val securityManager = new SecurityManager(conf); + val securityManager = new SecurityManager(conf) val hostname = "localhost" val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, @@ -135,12 +138,12 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") val goodconf = new SparkConf goodconf.set("spark.authenticate", "true") goodconf.set("spark.authenticate.secret", "good") - val securityManagerGood = new SecurityManager(goodconf); + val securityManagerGood = new SecurityManager(goodconf) assert(securityManagerGood.isAuthenticationEnabled() === true) @@ -148,7 +151,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro conf = goodconf, securityManager = securityManagerGood) val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( - s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") + AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) val timeout = AkkaUtils.lookupTimeout(conf) slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) @@ -175,7 +178,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro conf.set("spark.authenticate", "true") conf.set("spark.authenticate.secret", "good") - val securityManager = new SecurityManager(conf); + val securityManager = new SecurityManager(conf) val hostname = "localhost" val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, @@ -186,12 +189,12 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") val badconf = new SparkConf badconf.set("spark.authenticate", "false") badconf.set("spark.authenticate.secret", "bad") - val securityManagerBad = new SecurityManager(badconf); + val securityManagerBad = new SecurityManager(badconf) assert(securityManagerBad.isAuthenticationEnabled() === false) @@ -199,7 +202,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro conf = badconf, securityManager = securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( - s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") + AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) val timeout = AkkaUtils.lookupTimeout(conf) intercept[akka.actor.ActorNotFound] { slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) @@ -209,4 +212,170 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro slaveSystem.shutdown() } + test("remote fetch ssl on") { + val conf = sparkSSLConfig() + val securityManager = new SecurityManager(conf) + + val hostname = "localhost" + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + conf = conf, securityManager = securityManager) + System.setProperty("spark.hostPort", hostname + ":" + boundPort) + + assert(securityManager.isAuthenticationEnabled() === false) + + val masterTracker = new MapOutputTrackerMaster(conf) + masterTracker.trackerActor = actorSystem.actorOf( + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + + val slaveConf = sparkSSLConfig() + val securityManagerBad = new SecurityManager(slaveConf) + + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, + conf = slaveConf, securityManager = securityManagerBad) + val slaveTracker = new MapOutputTrackerWorker(conf) + val selection = slaveSystem.actorSelection( + AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) + val timeout = AkkaUtils.lookupTimeout(conf) + slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + + assert(securityManagerBad.isAuthenticationEnabled() === false) + + masterTracker.registerShuffle(10, 1) + masterTracker.incrementEpoch() + slaveTracker.updateEpoch(masterTracker.getEpoch) + + val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) + masterTracker.registerMapOutput(10, 0, + MapStatus(BlockManagerId("a", "hostA", 1000), Array(1000L))) + masterTracker.incrementEpoch() + slaveTracker.updateEpoch(masterTracker.getEpoch) + + // this should succeed since security off + assert(slaveTracker.getServerStatuses(10, 0).toSeq === + Seq((BlockManagerId("a", "hostA", 1000), size1000))) + + actorSystem.shutdown() + slaveSystem.shutdown() + } + + + test("remote fetch ssl on and security enabled") { + val conf = sparkSSLConfig() + conf.set("spark.authenticate", "true") + conf.set("spark.authenticate.secret", "good") + val securityManager = new SecurityManager(conf) + + val hostname = "localhost" + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + conf = conf, securityManager = securityManager) + System.setProperty("spark.hostPort", hostname + ":" + boundPort) + + assert(securityManager.isAuthenticationEnabled() === true) + + val masterTracker = new MapOutputTrackerMaster(conf) + masterTracker.trackerActor = actorSystem.actorOf( + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + + val slaveConf = sparkSSLConfig() + slaveConf.set("spark.authenticate", "true") + slaveConf.set("spark.authenticate.secret", "good") + val securityManagerBad = new SecurityManager(slaveConf) + + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, + conf = slaveConf, securityManager = securityManagerBad) + val slaveTracker = new MapOutputTrackerWorker(conf) + val selection = slaveSystem.actorSelection( + AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) + val timeout = AkkaUtils.lookupTimeout(conf) + slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + + assert(securityManagerBad.isAuthenticationEnabled() === true) + + masterTracker.registerShuffle(10, 1) + masterTracker.incrementEpoch() + slaveTracker.updateEpoch(masterTracker.getEpoch) + + val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) + masterTracker.registerMapOutput(10, 0, + MapStatus(BlockManagerId("a", "hostA", 1000), Array(1000L))) + masterTracker.incrementEpoch() + slaveTracker.updateEpoch(masterTracker.getEpoch) + + assert(slaveTracker.getServerStatuses(10, 0).toSeq === + Seq((BlockManagerId("a", "hostA", 1000), size1000))) + + actorSystem.shutdown() + slaveSystem.shutdown() + } + + + test("remote fetch ssl on and security enabled - bad credentials") { + val conf = sparkSSLConfig() + conf.set("spark.authenticate", "true") + conf.set("spark.authenticate.secret", "good") + val securityManager = new SecurityManager(conf) + + val hostname = "localhost" + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + conf = conf, securityManager = securityManager) + System.setProperty("spark.hostPort", hostname + ":" + boundPort) + + assert(securityManager.isAuthenticationEnabled() === true) + + val masterTracker = new MapOutputTrackerMaster(conf) + masterTracker.trackerActor = actorSystem.actorOf( + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + + val slaveConf = sparkSSLConfig() + slaveConf.set("spark.authenticate", "true") + slaveConf.set("spark.authenticate.secret", "bad") + val securityManagerBad = new SecurityManager(slaveConf) + + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, + conf = slaveConf, securityManager = securityManagerBad) + val slaveTracker = new MapOutputTrackerWorker(conf) + val selection = slaveSystem.actorSelection( + AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) + val timeout = AkkaUtils.lookupTimeout(conf) + intercept[akka.actor.ActorNotFound] { + slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + } + + actorSystem.shutdown() + slaveSystem.shutdown() + } + + + test("remote fetch ssl on - untrusted server") { + val conf = sparkSSLConfigUntrusted() + val securityManager = new SecurityManager(conf) + + val hostname = "localhost" + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + conf = conf, securityManager = securityManager) + System.setProperty("spark.hostPort", hostname + ":" + boundPort) + + assert(securityManager.isAuthenticationEnabled() === false) + + val masterTracker = new MapOutputTrackerMaster(conf) + masterTracker.trackerActor = actorSystem.actorOf( + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + + val slaveConf = sparkSSLConfig() + val securityManagerBad = new SecurityManager(slaveConf) + + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, + conf = slaveConf, securityManager = securityManagerBad) + val slaveTracker = new MapOutputTrackerWorker(conf) + val selection = slaveSystem.actorSelection( + AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) + val timeout = AkkaUtils.lookupTimeout(conf) + intercept[TimeoutException] { + slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + } + + actorSystem.shutdown() + slaveSystem.shutdown() + } + } diff --git a/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala b/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala index d4b92f33dd9e6..bad1aa99952cf 100644 --- a/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala +++ b/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala @@ -19,6 +19,7 @@ package org.apache.spark.util import java.util.Properties +import org.apache.commons.lang3.SerializationUtils import org.scalatest.{BeforeAndAfterEach, Suite} /** @@ -42,7 +43,11 @@ private[spark] trait ResetSystemProperties extends BeforeAndAfterEach { this: Su var oldProperties: Properties = null override def beforeEach(): Unit = { - oldProperties = new Properties(System.getProperties) + // we need SerializationUtils.clone instead of `new Properties(System.getProperties()` because + // the later way of creating a copy does not copy the properties but it initializes a new + // Properties object with the given properties as defaults. They are not recognized at all + // by standard Scala wrapper over Java Properties then. + oldProperties = SerializationUtils.clone(System.getProperties) super.beforeEach() } diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index b2a7e092a0291..607ce1c803507 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -22,8 +22,9 @@ # Expects to be run in a totally empty directory. # # Options: -# --package-only only packages an existing release candidate -# +# --skip-create-release Assume the desired release tag already exists +# --skip-publish Do not publish to Maven central +# --skip-package Do not package and upload binary artifacts # Would be nice to add: # - Send output to stderr and have useful logging in stdout @@ -51,7 +52,7 @@ set -e GIT_TAG=v$RELEASE_VERSION-$RC_NAME -if [[ ! "$@" =~ --package-only ]]; then +if [[ ! "$@" =~ --skip-create-release ]]; then echo "Creating release commit and publishing to Apache repository" # Artifact publishing git clone https://$ASF_USERNAME:$ASF_PASSWORD@git-wip-us.apache.org/repos/asf/spark.git \ @@ -87,8 +88,15 @@ if [[ ! "$@" =~ --package-only ]]; then git commit -a -m "Preparing development version $next_ver" git push origin $GIT_TAG git push origin HEAD:$GIT_BRANCH - git checkout -f $GIT_TAG + popd + rm -rf spark +fi +if [[ ! "$@" =~ --skip-publish ]]; then + git clone https://$ASF_USERNAME:$ASF_PASSWORD@git-wip-us.apache.org/repos/asf/spark.git + pushd spark + git checkout --force $GIT_TAG + # Using Nexus API documented here: # https://support.sonatype.com/entries/39720203-Uploading-to-a-Staging-Repository-via-REST-API echo "Creating Nexus staging repository" @@ -106,7 +114,7 @@ if [[ ! "$@" =~ --package-only ]]; then clean install ./dev/change-version-to-2.11.sh - + mvn -DskipTests -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ -Dscala-2.11 -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ clean install @@ -155,88 +163,90 @@ if [[ ! "$@" =~ --package-only ]]; then rm -rf spark fi -# Source and binary tarballs -echo "Packaging release tarballs" -git clone https://git-wip-us.apache.org/repos/asf/spark.git -cd spark -git checkout --force $GIT_TAG -release_hash=`git rev-parse HEAD` - -rm .gitignore -rm -rf .git -cd .. - -cp -r spark spark-$RELEASE_VERSION -tar cvzf spark-$RELEASE_VERSION.tgz spark-$RELEASE_VERSION -echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --armour --output spark-$RELEASE_VERSION.tgz.asc \ - --detach-sig spark-$RELEASE_VERSION.tgz -echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md MD5 spark-$RELEASE_VERSION.tgz > \ - spark-$RELEASE_VERSION.tgz.md5 -echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md SHA512 spark-$RELEASE_VERSION.tgz > \ - spark-$RELEASE_VERSION.tgz.sha -rm -rf spark-$RELEASE_VERSION - -make_binary_release() { - NAME=$1 - FLAGS=$2 - cp -r spark spark-$RELEASE_VERSION-bin-$NAME - - cd spark-$RELEASE_VERSION-bin-$NAME - - # TODO There should probably be a flag to make-distribution to allow 2.11 support - if [[ $FLAGS == *scala-2.11* ]]; then - ./dev/change-version-to-2.11.sh - fi - - ./make-distribution.sh --name $NAME --tgz $FLAGS 2>&1 | tee ../binary-release-$NAME.log +if [[ ! "$@" =~ --skip-package ]]; then + # Source and binary tarballs + echo "Packaging release tarballs" + git clone https://git-wip-us.apache.org/repos/asf/spark.git + cd spark + git checkout --force $GIT_TAG + release_hash=`git rev-parse HEAD` + + rm .gitignore + rm -rf .git cd .. - cp spark-$RELEASE_VERSION-bin-$NAME/spark-$RELEASE_VERSION-bin-$NAME.tgz . - rm -rf spark-$RELEASE_VERSION-bin-$NAME - - echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --armour \ - --output spark-$RELEASE_VERSION-bin-$NAME.tgz.asc \ - --detach-sig spark-$RELEASE_VERSION-bin-$NAME.tgz - echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md \ - MD5 spark-$RELEASE_VERSION-bin-$NAME.tgz > \ - spark-$RELEASE_VERSION-bin-$NAME.tgz.md5 - echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md \ - SHA512 spark-$RELEASE_VERSION-bin-$NAME.tgz > \ - spark-$RELEASE_VERSION-bin-$NAME.tgz.sha -} - - -make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" & -make_binary_release "hadoop1-scala2.11" "-Phive -Dscala-2.11" & -make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" & -make_binary_release "hadoop2.3" "-Phadoop-2.3 -Phive -Phive-thriftserver -Pyarn" & -make_binary_release "hadoop2.4" "-Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn" & -make_binary_release "mapr3" "-Pmapr3 -Phive -Phive-thriftserver" & -make_binary_release "mapr4" "-Pmapr4 -Pyarn -Phive -Phive-thriftserver" & -make_binary_release "hadoop2.4-without-hive" "-Phadoop-2.4 -Pyarn" & -wait - -# Copy data -echo "Copying release tarballs" -rc_folder=spark-$RELEASE_VERSION-$RC_NAME -ssh $ASF_USERNAME@people.apache.org \ - mkdir /home/$ASF_USERNAME/public_html/$rc_folder -scp spark-* \ - $ASF_USERNAME@people.apache.org:/home/$ASF_USERNAME/public_html/$rc_folder/ - -# Docs -cd spark -build/sbt clean -cd docs -# Compile docs with Java 7 to use nicer format -JAVA_HOME=$JAVA_7_HOME PRODUCTION=1 jekyll build -echo "Copying release documentation" -rc_docs_folder=${rc_folder}-docs -ssh $ASF_USERNAME@people.apache.org \ - mkdir /home/$ASF_USERNAME/public_html/$rc_docs_folder -rsync -r _site/* $ASF_USERNAME@people.apache.org:/home/$ASF_USERNAME/public_html/$rc_docs_folder - -echo "Release $RELEASE_VERSION completed:" -echo "Git tag:\t $GIT_TAG" -echo "Release commit:\t $release_hash" -echo "Binary location:\t http://people.apache.org/~$ASF_USERNAME/$rc_folder" -echo "Doc location:\t http://people.apache.org/~$ASF_USERNAME/$rc_docs_folder" + + cp -r spark spark-$RELEASE_VERSION + tar cvzf spark-$RELEASE_VERSION.tgz spark-$RELEASE_VERSION + echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --armour --output spark-$RELEASE_VERSION.tgz.asc \ + --detach-sig spark-$RELEASE_VERSION.tgz + echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md MD5 spark-$RELEASE_VERSION.tgz > \ + spark-$RELEASE_VERSION.tgz.md5 + echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md SHA512 spark-$RELEASE_VERSION.tgz > \ + spark-$RELEASE_VERSION.tgz.sha + rm -rf spark-$RELEASE_VERSION + + make_binary_release() { + NAME=$1 + FLAGS=$2 + cp -r spark spark-$RELEASE_VERSION-bin-$NAME + + cd spark-$RELEASE_VERSION-bin-$NAME + + # TODO There should probably be a flag to make-distribution to allow 2.11 support + if [[ $FLAGS == *scala-2.11* ]]; then + ./dev/change-version-to-2.11.sh + fi + + ./make-distribution.sh --name $NAME --tgz $FLAGS 2>&1 | tee ../binary-release-$NAME.log + cd .. + cp spark-$RELEASE_VERSION-bin-$NAME/spark-$RELEASE_VERSION-bin-$NAME.tgz . + rm -rf spark-$RELEASE_VERSION-bin-$NAME + + echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --armour \ + --output spark-$RELEASE_VERSION-bin-$NAME.tgz.asc \ + --detach-sig spark-$RELEASE_VERSION-bin-$NAME.tgz + echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md \ + MD5 spark-$RELEASE_VERSION-bin-$NAME.tgz > \ + spark-$RELEASE_VERSION-bin-$NAME.tgz.md5 + echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md \ + SHA512 spark-$RELEASE_VERSION-bin-$NAME.tgz > \ + spark-$RELEASE_VERSION-bin-$NAME.tgz.sha + } + + + make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" & + make_binary_release "hadoop1-scala2.11" "-Phive -Dscala-2.11" & + make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" & + make_binary_release "hadoop2.3" "-Phadoop-2.3 -Phive -Phive-thriftserver -Pyarn" & + make_binary_release "hadoop2.4" "-Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn" & + make_binary_release "mapr3" "-Pmapr3 -Phive -Phive-thriftserver" & + make_binary_release "mapr4" "-Pmapr4 -Pyarn -Phive -Phive-thriftserver" & + make_binary_release "hadoop2.4-without-hive" "-Phadoop-2.4 -Pyarn" & + wait + + # Copy data + echo "Copying release tarballs" + rc_folder=spark-$RELEASE_VERSION-$RC_NAME + ssh $ASF_USERNAME@people.apache.org \ + mkdir /home/$ASF_USERNAME/public_html/$rc_folder + scp spark-* \ + $ASF_USERNAME@people.apache.org:/home/$ASF_USERNAME/public_html/$rc_folder/ + + # Docs + cd spark + sbt/sbt clean + cd docs + # Compile docs with Java 7 to use nicer format + JAVA_HOME=$JAVA_7_HOME PRODUCTION=1 jekyll build + echo "Copying release documentation" + rc_docs_folder=${rc_folder}-docs + ssh $ASF_USERNAME@people.apache.org \ + mkdir /home/$ASF_USERNAME/public_html/$rc_docs_folder + rsync -r _site/* $ASF_USERNAME@people.apache.org:/home/$ASF_USERNAME/public_html/$rc_docs_folder + + echo "Release $RELEASE_VERSION completed:" + echo "Git tag:\t $GIT_TAG" + echo "Release commit:\t $release_hash" + echo "Binary location:\t http://people.apache.org/~$ASF_USERNAME/$rc_folder" + echo "Doc location:\t http://people.apache.org/~$ASF_USERNAME/$rc_docs_folder" +fi diff --git a/docs/building-spark.md b/docs/building-spark.md index fb93017861ed0..db69905813817 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -137,15 +137,18 @@ We use the scala-maven-plugin which supports incremental and continuous compilat should run continuous compilation (i.e. wait for changes). However, this has not been tested extensively. A couple of gotchas to note: + * it only scans the paths `src/main` and `src/test` (see [docs](http://scala-tools.org/mvnsites/maven-scala-plugin/usage_cc.html)), so it will only work from within certain submodules that have that structure. + * you'll typically need to run `mvn install` from the project root for compilation within specific submodules to work; this is because submodules that depend on other submodules do so via the `spark-parent` module). Thus, the full flow for running continuous-compilation of the `core` submodule may look more like: - ``` + +``` $ mvn install $ cd core $ mvn scala:cc diff --git a/docs/configuration.md b/docs/configuration.md index e4e4b8d516b75..62d3fca937b2c 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1098,24 +1098,32 @@ Apart from these, the following properties are also available, and may be useful available on YARN mode. For more detail, see the description here.

- This requires the following configurations to be set: + This requires spark.shuffle.service.enabled to be set. + The following configurations are also relevant: spark.dynamicAllocation.minExecutors, spark.dynamicAllocation.maxExecutors, and - spark.shuffle.service.enabled + spark.dynamicAllocation.initialExecutors spark.dynamicAllocation.minExecutors - (none) + 0 - Lower bound for the number of executors if dynamic allocation is enabled (required). + Lower bound for the number of executors if dynamic allocation is enabled. spark.dynamicAllocation.maxExecutors - (none) + Integer.MAX_VALUE + + Upper bound for the number of executors if dynamic allocation is enabled. + + + + spark.dynamicAllocation.maxExecutors + spark.dynamicAllocation.minExecutors - Upper bound for the number of executors if dynamic allocation is enabled (required). + Initial number of executors to run if dynamic allocation is enabled. @@ -1234,6 +1242,86 @@ Apart from these, the following properties are also available, and may be useful +#### Encryption + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Property NameDefaultMeaning
spark.ssl.enabledfalse +

Whether to enable SSL connections on all supported protocols.

+ +

All the SSL settings like spark.ssl.xxx where xxx is a + particular configuration property, denote the global configuration for all the supported + protocols. In order to override the global configuration for the particular protocol, + the properties must be overwritten in the protocol-specific namespace.

+ +

Use spark.ssl.YYY.XXX settings to overwrite the global configuration for + particular protocol denoted by YYY. Currently YYY can be + either akka for Akka based connections or fs for broadcast and + file server.

+
spark.ssl.keyStoreNone + A path to a key-store file. The path can be absolute or relative to the directory where + the component is started in. +
spark.ssl.keyStorePasswordNone + A password to the key-store. +
spark.ssl.keyPasswordNone + A password to the private key in key-store. +
spark.ssl.trustStoreNone + A path to a trust-store file. The path can be absolute or relative to the directory + where the component is started in. +
spark.ssl.trustStorePasswordNone + A password to the trust-store. +
spark.ssl.protocolNone + A protocol name. The protocol must be supported by JVM. The reference list of protocols + one can find on this + page. +
spark.ssl.enabledAlgorithmsEmpty + A comma separated list of ciphers. The specified ciphers must be supported by JVM. + The reference list of protocols one can find on + this + page. +
+ + #### Spark Streaming diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md index a5425eb3557b2..5295e351dd711 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -77,11 +77,10 @@ scheduling while sharing cluster resources efficiently. ### Configuration and Setup All configurations used by this feature live under the `spark.dynamicAllocation.*` namespace. -To enable this feature, your application must set `spark.dynamicAllocation.enabled` to `true` and -provide lower and upper bounds for the number of executors through -`spark.dynamicAllocation.minExecutors` and `spark.dynamicAllocation.maxExecutors`. Other relevant -configurations are described on the [configurations page](configuration.html#dynamic-allocation) -and in the subsequent sections in detail. +To enable this feature, your application must set `spark.dynamicAllocation.enabled` to `true`. +Other relevant configurations are described on the +[configurations page](configuration.html#dynamic-allocation) and in the subsequent sections in +detail. Additionally, your application must use an external shuffle service. The purpose of the service is to preserve the shuffle files written by executors so the executors can be safely removed (more diff --git a/docs/security.md b/docs/security.md index 1e206a139fb72..6e0a54fbc4ad7 100644 --- a/docs/security.md +++ b/docs/security.md @@ -20,6 +20,30 @@ Spark allows for a set of administrators to be specified in the acls who always If your applications are using event logging, the directory where the event logs go (`spark.eventLog.dir`) should be manually created and have the proper permissions set on it. If you want those log files secured, the permissions should be set to `drwxrwxrwxt` for that directory. The owner of the directory should be the super user who is running the history server and the group permissions should be restricted to super user group. This will allow all users to write to the directory but will prevent unprivileged users from removing or renaming a file unless they own the file or directory. The event log files will be created by Spark with permissions such that only the user and group have read and write access. +## Encryption + +Spark supports SSL for Akka and HTTP (for broadcast and file server) protocols. However SSL is not supported yet for WebUI and block transfer service. + +Connection encryption (SSL) configuration is organized hierarchically. The user can configure the default SSL settings which will be used for all the supported communication protocols unless they are overwritten by protocol-specific settings. This way the user can easily provide the common settings for all the protocols without disabling the ability to configure each one individually. The common SSL settings are at `spark.ssl` namespace in Spark configuration, while Akka SSL configuration is at `spark.ssl.akka` and HTTP for broadcast and file server SSL configuration is at `spark.ssl.fs`. The full breakdown can be found on the [configuration page](configuration.html). + +SSL must be configured on each node and configured for each component involved in communication using the particular protocol. + +### YARN mode +The key-store can be prepared on the client side and then distributed and used by the executors as the part of the application. It is possible because the user is able to deploy files before the application is started in YARN by using `spark.yarn.dist.files` or `spark.yarn.dist.archives` configuration settings. The responsibility for encryption of transferring these files is on YARN side and has nothing to do with Spark. + +### Standalone mode +The user needs to provide key-stores and configuration options for master and workers. They have to be set by attaching appropriate Java system properties in `SPARK_MASTER_OPTS` and in `SPARK_WORKER_OPTS` environment variables, or just in `SPARK_DAEMON_JAVA_OPTS`. In this mode, the user may allow the executors to use the SSL settings inherited from the worker which spawned that executor. It can be accomplished by setting `spark.ssl.useNodeLocalConf` to `true`. If that parameter is set, the settings provided by user on the client side, are not used by the executors. + +### Preparing the key-stores +Key-stores can be generated by `keytool` program. The reference documentation for this tool is +[here](https://docs.oracle.com/javase/7/docs/technotes/tools/solaris/keytool.html). The most basic +steps to configure the key-stores and the trust-store for the standalone deployment mode is as +follows: +* Generate a keys pair for each node +* Export the public key of the key pair to a file on each node +* Import all exported public keys into a single trust-store +* Distribute the trust-store over the nodes + ## Configuring Ports for Network Security Spark makes heavy use of the network, and some environments have strict requirements for using tight diff --git a/examples/src/main/python/streaming/kafka_wordcount.py b/examples/src/main/python/streaming/kafka_wordcount.py new file mode 100644 index 0000000000000..ed398a82b8bb0 --- /dev/null +++ b/examples/src/main/python/streaming/kafka_wordcount.py @@ -0,0 +1,54 @@ +# +# 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. +# + +""" + Counts words in UTF8 encoded, '\n' delimited text received from the network every second. + Usage: network_wordcount.py + + To run this on your local machine, you need to setup Kafka and create a producer first, see + http://kafka.apache.org/documentation.html#quickstart + + and then run the example + `$ bin/spark-submit --driver-class-path external/kafka-assembly/target/scala-*/\ + spark-streaming-kafka-assembly-*.jar examples/src/main/python/streaming/kafka_wordcount.py \ + localhost:2181 test` +""" + +import sys + +from pyspark import SparkContext +from pyspark.streaming import StreamingContext +from pyspark.streaming.kafka import KafkaUtils + +if __name__ == "__main__": + if len(sys.argv) != 3: + print >> sys.stderr, "Usage: kafka_wordcount.py " + exit(-1) + + sc = SparkContext(appName="PythonStreamingKafkaWordCount") + ssc = StreamingContext(sc, 1) + + zkQuorum, topic = sys.argv[1:] + kvs = KafkaUtils.createStream(ssc, zkQuorum, "spark-streaming-consumer", {topic: 1}) + lines = kvs.map(lambda x: x[1]) + counts = lines.flatMap(lambda line: line.split(" ")) \ + .map(lambda word: (word, 1)) \ + .reduceByKey(lambda a, b: a+b) + counts.pprint() + + ssc.start() + ssc.awaitTermination() diff --git a/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala b/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala index 65251e93190f0..e757283823fc3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala @@ -19,6 +19,8 @@ package org.apache.spark.examples import scala.collection.JavaConversions._ +import org.apache.spark.util.Utils + /** Prints out environmental information, sleeps, and then exits. Made to * test driver submission in the standalone scheduler. */ object DriverSubmissionTest { @@ -30,7 +32,7 @@ object DriverSubmissionTest { val numSecondsToSleep = args(0).toInt val env = System.getenv() - val properties = System.getProperties() + val properties = Utils.getSystemProperties println("Environment variables containing SPARK_TEST:") env.filter{case (k, v) => k.contains("SPARK_TEST")}.foreach(println) diff --git a/external/kafka-assembly/pom.xml b/external/kafka-assembly/pom.xml new file mode 100644 index 0000000000000..503fc129dc4f2 --- /dev/null +++ b/external/kafka-assembly/pom.xml @@ -0,0 +1,106 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent + 1.3.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-streaming-kafka-assembly_2.10 + jar + Spark Project External Kafka Assembly + http://spark.apache.org/ + + + streaming-kafka-assembly + scala-${scala.binary.version} + spark-streaming-kafka-assembly-${project.version}.jar + ${project.build.directory}/${spark.jar.dir}/${spark.jar.basename} + + + + + org.apache.spark + spark-streaming-kafka_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + provided + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.apache.maven.plugins + maven-shade-plugin + + false + ${spark.jar} + + + *:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + package + + shade + + + + + + reference.conf + + + log4j.properties + + + + + + + + + + + + diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala index 77661f71ada21..1ef91dd49284f 100644 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala @@ -55,14 +55,14 @@ class MQTTInputDStream( brokerUrl: String, topic: String, storageLevel: StorageLevel - ) extends ReceiverInputDStream[String](ssc_) with Logging { - + ) extends ReceiverInputDStream[String](ssc_) { + def getReceiver(): Receiver[String] = { new MQTTReceiver(brokerUrl, topic, storageLevel) } } -private[streaming] +private[streaming] class MQTTReceiver( brokerUrl: String, topic: String, @@ -72,21 +72,15 @@ class MQTTReceiver( def onStop() { } - + def onStart() { - // Set up persistence for messages + // Set up persistence for messages val persistence = new MemoryPersistence() // Initializing Mqtt Client specifying brokerUrl, clientID and MqttClientPersistance val client = new MqttClient(brokerUrl, MqttClient.generateClientId(), persistence) - // Connect to MqttBroker - client.connect() - - // Subscribe to Mqtt topic - client.subscribe(topic) - // Callback automatically triggers as and when new message arrives on specified topic val callback: MqttCallback = new MqttCallback() { @@ -103,7 +97,15 @@ class MQTTReceiver( } } - // Set up callback for MqttClient + // Set up callback for MqttClient. This needs to happen before + // connecting or subscribing, otherwise messages may be lost client.setCallback(callback) + + // Connect to MqttBroker + client.connect() + + // Subscribe to Mqtt topic + client.subscribe(topic) + } } diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala index fe53a29cba0c9..e84adc088a680 100644 --- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala +++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala @@ -18,6 +18,8 @@ package org.apache.spark.streaming.mqtt import java.net.{URI, ServerSocket} +import java.util.concurrent.CountDownLatch +import java.util.concurrent.TimeUnit import scala.concurrent.duration._ import scala.language.postfixOps @@ -32,6 +34,8 @@ import org.scalatest.concurrent.Eventually import org.apache.spark.streaming.{Milliseconds, StreamingContext} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.ReceiverInputDStream +import org.apache.spark.streaming.scheduler.StreamingListener +import org.apache.spark.streaming.scheduler.StreamingListenerReceiverStarted import org.apache.spark.SparkConf import org.apache.spark.util.Utils @@ -67,7 +71,7 @@ class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter { val sendMessage = "MQTT demo for spark streaming" val receiveStream: ReceiverInputDStream[String] = MQTTUtils.createStream(ssc, "tcp:" + brokerUri, topic, StorageLevel.MEMORY_ONLY) - var receiveMessage: List[String] = List() + @volatile var receiveMessage: List[String] = List() receiveStream.foreachRDD { rdd => if (rdd.collect.length > 0) { receiveMessage = receiveMessage ::: List(rdd.first) @@ -75,6 +79,11 @@ class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter { } } ssc.start() + + // wait for the receiver to start before publishing data, or we risk failing + // the test nondeterministically. See SPARK-4631 + waitForReceiverToStart() + publishData(sendMessage) eventually(timeout(10000 milliseconds), interval(100 milliseconds)) { assert(sendMessage.equals(receiveMessage(0))) @@ -121,8 +130,14 @@ class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter { val message: MqttMessage = new MqttMessage(data.getBytes("utf-8")) message.setQos(1) message.setRetained(true) - for (i <- 0 to 100) { - msgTopic.publish(message) + + for (i <- 0 to 10) { + try { + msgTopic.publish(message) + } catch { + case e: MqttException if e.getReasonCode == MqttException.REASON_CODE_MAX_INFLIGHT => + Thread.sleep(50) // wait for Spark streaming to consume something from the message queue + } } } } finally { @@ -131,4 +146,18 @@ class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter { client = null } } + + /** + * Block until at least one receiver has started or timeout occurs. + */ + private def waitForReceiverToStart() = { + val latch = new CountDownLatch(1) + ssc.addStreamingListener(new StreamingListener { + override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) { + latch.countDown() + } + }) + + assert(latch.await(10, TimeUnit.SECONDS), "Timeout waiting for receiver to start.") + } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index ab56580a3abc8..8494d06b1cdb7 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -104,6 +104,18 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab */ def checkpoint(): Unit + /** + * Return whether this Graph has been checkpointed or not. + * This returns true iff both the vertices RDD and edges RDD have been checkpointed. + */ + def isCheckpointed: Boolean + + /** + * Gets the name of the files to which this Graph was checkpointed. + * (The vertices RDD and edges RDD are checkpointed separately.) + */ + def getCheckpointFiles: Seq[String] + /** * Uncaches both vertices and edges of this graph. This is useful in iterative algorithms that * build a new graph in each iteration. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala index f1550ac2e18ad..56cb41661e300 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala @@ -70,10 +70,20 @@ class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] ( this } + override def getStorageLevel = partitionsRDD.getStorageLevel + override def checkpoint() = { partitionsRDD.checkpoint() } - + + override def isCheckpointed: Boolean = { + firstParent[(PartitionID, EdgePartition[ED, VD])].isCheckpointed + } + + override def getCheckpointFile: Option[String] = { + partitionsRDD.getCheckpointFile + } + /** The number of edges in the RDD. */ override def count(): Long = { partitionsRDD.map(_._2.size.toLong).reduce(_ + _) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala index 3f4a900d5b601..90a74d23a26cc 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -70,6 +70,17 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( replicatedVertexView.edges.checkpoint() } + override def isCheckpointed: Boolean = { + vertices.isCheckpointed && replicatedVertexView.edges.isCheckpointed + } + + override def getCheckpointFiles: Seq[String] = { + Seq(vertices.getCheckpointFile, replicatedVertexView.edges.getCheckpointFile).flatMap { + case Some(path) => Seq(path) + case None => Seq() + } + } + override def unpersist(blocking: Boolean = true): Graph[VD, ED] = { unpersistVertices(blocking) replicatedVertexView.edges.unpersist(blocking) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala index 9732c5b00c6d9..6dad167fa7411 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala @@ -71,10 +71,20 @@ class VertexRDDImpl[VD] private[graphx] ( this } + override def getStorageLevel = partitionsRDD.getStorageLevel + override def checkpoint() = { partitionsRDD.checkpoint() } - + + override def isCheckpointed: Boolean = { + firstParent[ShippableVertexPartition[VD]].isCheckpointed + } + + override def getCheckpointFile: Option[String] = { + partitionsRDD.getCheckpointFile + } + /** The number of vertices in the RDD. */ override def count(): Long = { partitionsRDD.map(_.size).reduce(_ + _) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala new file mode 100644 index 0000000000000..eb1dbe52c2fda --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala @@ -0,0 +1,37 @@ +/* + * 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.graphx + +import org.scalatest.FunSuite + +import org.apache.spark.storage.StorageLevel + +class EdgeRDDSuite extends FunSuite with LocalSparkContext { + + test("cache, getStorageLevel") { + // test to see if getStorageLevel returns correct value after caching + withSpark { sc => + val verts = sc.parallelize(List((0L, 0), (1L, 1), (1L, 2), (2L, 3), (2L, 3), (2L, 3))) + val edges = EdgeRDD.fromEdges(sc.parallelize(List.empty[Edge[Int]])) + assert(edges.getStorageLevel == StorageLevel.NONE) + edges.cache() + assert(edges.getStorageLevel == StorageLevel.MEMORY_ONLY) + } + } + +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index ed9876b8dc21c..b61d9f0fbe5e4 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -25,6 +25,7 @@ import org.apache.spark.SparkContext import org.apache.spark.graphx.Graph._ import org.apache.spark.graphx.PartitionStrategy._ import org.apache.spark.rdd._ +import org.apache.spark.storage.StorageLevel class GraphSuite extends FunSuite with LocalSparkContext { @@ -375,6 +376,8 @@ class GraphSuite extends FunSuite with LocalSparkContext { val ring = (0L to 100L).zip((1L to 99L) :+ 0L).map { case (a, b) => Edge(a, b, 1)} val rdd = sc.parallelize(ring) val graph = Graph.fromEdges(rdd, 1.0F) + assert(!graph.isCheckpointed) + assert(graph.getCheckpointFiles.size === 0) graph.checkpoint() graph.edges.map(_.attr).count() graph.vertices.map(_._2).count() @@ -383,6 +386,22 @@ class GraphSuite extends FunSuite with LocalSparkContext { val verticesDependencies = graph.vertices.partitionsRDD.dependencies assert(edgesDependencies.forall(_.rdd.isInstanceOf[CheckpointRDD[_]])) assert(verticesDependencies.forall(_.rdd.isInstanceOf[CheckpointRDD[_]])) + assert(graph.isCheckpointed) + assert(graph.getCheckpointFiles.size === 2) + } + } + + test("cache, getStorageLevel") { + // test to see if getStorageLevel returns correct value + withSpark { sc => + val verts = sc.parallelize(List((1: VertexId, "a"), (2: VertexId, "b")), 1) + val edges = sc.parallelize(List(Edge(1, 2, 0), Edge(2, 1, 0)), 2) + val graph = Graph(verts, edges, "", StorageLevel.MEMORY_ONLY, StorageLevel.MEMORY_ONLY) + // Note: Before caching, graph.vertices is cached, but graph.edges is not (but graph.edges' + // parent RDD is cached). + graph.cache() + assert(graph.vertices.getStorageLevel == StorageLevel.MEMORY_ONLY) + assert(graph.edges.getStorageLevel == StorageLevel.MEMORY_ONLY) } } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala index 42d3f21dbae98..131959cea3ef7 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala @@ -17,12 +17,11 @@ package org.apache.spark.graphx -import org.apache.spark.SparkContext -import org.apache.spark.graphx.Graph._ -import org.apache.spark.graphx.impl.EdgePartition -import org.apache.spark.rdd._ import org.scalatest.FunSuite +import org.apache.spark.SparkContext +import org.apache.spark.storage.StorageLevel + class VertexRDDSuite extends FunSuite with LocalSparkContext { def vertices(sc: SparkContext, n: Int) = { @@ -110,4 +109,16 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { } } + test("cache, getStorageLevel") { + // test to see if getStorageLevel returns correct value after caching + withSpark { sc => + val verts = sc.parallelize(List((0L, 0), (1L, 1), (1L, 2), (2L, 3), (2L, 3), (2L, 3))) + val edges = EdgeRDD.fromEdges(sc.parallelize(List.empty[Edge[Int]])) + val rdd = VertexRDD(verts, edges, 0, (a: Int, b: Int) => a + b) + assert(rdd.getStorageLevel == StorageLevel.NONE) + rdd.cache() + assert(rdd.getStorageLevel == StorageLevel.MEMORY_ONLY) + } + } + } diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index 979a19d3b2057..82d21d5e4cb6e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -25,12 +25,14 @@ import scala.util.Sorting import com.github.fommil.netlib.BLAS.{getInstance => blas} import com.github.fommil.netlib.LAPACK.{getInstance => lapack} +import org.jblas.DoubleMatrix import org.netlib.util.intW import org.apache.spark.{HashPartitioner, Logging, Partitioner} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.param._ +import org.apache.spark.mllib.optimization.NNLS import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame import org.apache.spark.sql.Dsl._ @@ -80,6 +82,10 @@ private[recommendation] trait ALSParams extends Params with HasMaxIter with HasR val ratingCol = new Param[String](this, "ratingCol", "column name for ratings", Some("rating")) def getRatingCol: String = get(ratingCol) + val nonnegative = new BooleanParam( + this, "nonnegative", "whether to use nonnegative constraint for least squares", Some(false)) + val getNonnegative: Boolean = get(nonnegative) + /** * Validates and transforms the input schema. * @param schema input schema @@ -186,6 +192,7 @@ class ALS extends Estimator[ALSModel] with ALSParams { def setPredictionCol(value: String): this.type = set(predictionCol, value) def setMaxIter(value: Int): this.type = set(maxIter, value) def setRegParam(value: Double): this.type = set(regParam, value) + def setNonnegative(value: Boolean): this.type = set(nonnegative, value) /** Sets both numUserBlocks and numItemBlocks to the specific value. */ def setNumBlocks(value: Int): this.type = { @@ -207,7 +214,7 @@ class ALS extends Estimator[ALSModel] with ALSParams { val (userFactors, itemFactors) = ALS.train(ratings, rank = map(rank), numUserBlocks = map(numUserBlocks), numItemBlocks = map(numItemBlocks), maxIter = map(maxIter), regParam = map(regParam), implicitPrefs = map(implicitPrefs), - alpha = map(alpha)) + alpha = map(alpha), nonnegative = map(nonnegative)) val model = new ALSModel(this, map, map(rank), userFactors, itemFactors) Params.inheritValues(map, this, model) model @@ -232,11 +239,16 @@ object ALS extends Logging { /** Rating class for better code readability. */ case class Rating[@specialized(Int, Long) ID](user: ID, item: ID, rating: Float) + /** Trait for least squares solvers applied to the normal equation. */ + private[recommendation] trait LeastSquaresNESolver extends Serializable { + /** Solves a least squares problem (possibly with other constraints). */ + def solve(ne: NormalEquation, lambda: Double): Array[Float] + } + /** Cholesky solver for least square problems. */ - private[recommendation] class CholeskySolver { + private[recommendation] class CholeskySolver extends LeastSquaresNESolver { private val upper = "U" - private val info = new intW(0) /** * Solves a least squares problem with L2 regularization: @@ -247,7 +259,7 @@ object ALS extends Logging { * @param lambda regularization constant, which will be scaled by n * @return the solution x */ - def solve(ne: NormalEquation, lambda: Double): Array[Float] = { + override def solve(ne: NormalEquation, lambda: Double): Array[Float] = { val k = ne.k // Add scaled lambda to the diagonals of AtA. val scaledlambda = lambda * ne.n @@ -258,6 +270,7 @@ object ALS extends Logging { i += j j += 1 } + val info = new intW(0) lapack.dppsv(upper, k, 1, ne.ata, ne.atb, k, info) val code = info.`val` assert(code == 0, s"lapack.dppsv returned $code.") @@ -272,6 +285,63 @@ object ALS extends Logging { } } + /** NNLS solver. */ + private[recommendation] class NNLSSolver extends LeastSquaresNESolver { + private var rank: Int = -1 + private var workspace: NNLS.Workspace = _ + private var ata: DoubleMatrix = _ + private var initialized: Boolean = false + + private def initialize(rank: Int): Unit = { + if (!initialized) { + this.rank = rank + workspace = NNLS.createWorkspace(rank) + ata = new DoubleMatrix(rank, rank) + initialized = true + } else { + require(this.rank == rank) + } + } + + /** + * Solves a nonnegative least squares problem with L2 regularizatin: + * + * min_x_ norm(A x - b)^2^ + lambda * n * norm(x)^2^ + * subject to x >= 0 + */ + override def solve(ne: NormalEquation, lambda: Double): Array[Float] = { + val rank = ne.k + initialize(rank) + fillAtA(ne.ata, lambda * ne.n) + val x = NNLS.solve(ata, new DoubleMatrix(rank, 1, ne.atb: _*), workspace) + ne.reset() + x.map(x => x.toFloat) + } + + /** + * Given a triangular matrix in the order of fillXtX above, compute the full symmetric square + * matrix that it represents, storing it into destMatrix. + */ + private def fillAtA(triAtA: Array[Double], lambda: Double) { + var i = 0 + var pos = 0 + var a = 0.0 + val data = ata.data + while (i < rank) { + var j = 0 + while (j <= i) { + a = triAtA(pos) + data(i * rank + j) = a + data(j * rank + i) = a + pos += 1 + j += 1 + } + data(i * rank + i) += lambda + i += 1 + } + } + } + /** Representing a normal equation (ALS' subproblem). */ private[recommendation] class NormalEquation(val k: Int) extends Serializable { @@ -350,12 +420,14 @@ object ALS extends Logging { maxIter: Int = 10, regParam: Double = 1.0, implicitPrefs: Boolean = false, - alpha: Double = 1.0)( + alpha: Double = 1.0, + nonnegative: Boolean = false)( implicit ord: Ordering[ID]): (RDD[(ID, Array[Float])], RDD[(ID, Array[Float])]) = { val userPart = new HashPartitioner(numUserBlocks) val itemPart = new HashPartitioner(numItemBlocks) val userLocalIndexEncoder = new LocalIndexEncoder(userPart.numPartitions) val itemLocalIndexEncoder = new LocalIndexEncoder(itemPart.numPartitions) + val solver = if (nonnegative) new NNLSSolver else new CholeskySolver val blockRatings = partitionRatings(ratings, userPart, itemPart).cache() val (userInBlocks, userOutBlocks) = makeBlocks("user", blockRatings, userPart, itemPart) // materialize blockRatings and user blocks @@ -374,20 +446,20 @@ object ALS extends Logging { userFactors.setName(s"userFactors-$iter").persist() val previousItemFactors = itemFactors itemFactors = computeFactors(userFactors, userOutBlocks, itemInBlocks, rank, regParam, - userLocalIndexEncoder, implicitPrefs, alpha) + userLocalIndexEncoder, implicitPrefs, alpha, solver) previousItemFactors.unpersist() itemFactors.setName(s"itemFactors-$iter").persist() val previousUserFactors = userFactors userFactors = computeFactors(itemFactors, itemOutBlocks, userInBlocks, rank, regParam, - itemLocalIndexEncoder, implicitPrefs, alpha) + itemLocalIndexEncoder, implicitPrefs, alpha, solver) previousUserFactors.unpersist() } } else { for (iter <- 0 until maxIter) { itemFactors = computeFactors(userFactors, userOutBlocks, itemInBlocks, rank, regParam, - userLocalIndexEncoder) + userLocalIndexEncoder, solver = solver) userFactors = computeFactors(itemFactors, itemOutBlocks, userInBlocks, rank, regParam, - itemLocalIndexEncoder) + itemLocalIndexEncoder, solver = solver) } } val userIdAndFactors = userInBlocks @@ -879,6 +951,7 @@ object ALS extends Logging { * @param srcEncoder encoder for src local indices * @param implicitPrefs whether to use implicit preference * @param alpha the alpha constant in the implicit preference formulation + * @param solver solver for least squares problems * * @return dst factors */ @@ -890,7 +963,8 @@ object ALS extends Logging { regParam: Double, srcEncoder: LocalIndexEncoder, implicitPrefs: Boolean = false, - alpha: Double = 1.0): RDD[(Int, FactorBlock)] = { + alpha: Double = 1.0, + solver: LeastSquaresNESolver): RDD[(Int, FactorBlock)] = { val numSrcBlocks = srcFactorBlocks.partitions.length val YtY = if (implicitPrefs) Some(computeYtY(srcFactorBlocks, rank)) else None val srcOut = srcOutBlocks.join(srcFactorBlocks).flatMap { @@ -909,7 +983,6 @@ object ALS extends Logging { val dstFactors = new Array[Array[Float]](dstIds.length) var j = 0 val ls = new NormalEquation(rank) - val solver = new CholeskySolver // TODO: add NNLS solver while (j < dstIds.length) { ls.reset() if (implicitPrefs) { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 94d757bc317ab..282fb3ff283f4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -18,30 +18,41 @@ package org.apache.spark.mllib.classification import org.apache.spark.annotation.Experimental -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.linalg.BLAS.dot +import org.apache.spark.mllib.linalg.{DenseVector, Vector} import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.DataValidators +import org.apache.spark.mllib.util.{DataValidators, MLUtils} import org.apache.spark.rdd.RDD /** - * Classification model trained using Logistic Regression. + * Classification model trained using Multinomial/Binary Logistic Regression. * * @param weights Weights computed for every feature. - * @param intercept Intercept computed for this model. + * @param intercept Intercept computed for this model. (Only used in Binary Logistic Regression. + * In Multinomial Logistic Regression, the intercepts will not be a single values, + * so the intercepts will be part of the weights.) + * @param numFeatures the dimension of the features. + * @param numClasses the number of possible outcomes for k classes classification problem in + * Multinomial Logistic Regression. By default, it is binary logistic regression + * so numClasses will be set to 2. */ class LogisticRegressionModel ( override val weights: Vector, - override val intercept: Double) + override val intercept: Double, + val numFeatures: Int, + val numClasses: Int) extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable { + def this(weights: Vector, intercept: Double) = this(weights, intercept, weights.size, 2) + private var threshold: Option[Double] = Some(0.5) /** * :: Experimental :: - * Sets the threshold that separates positive predictions from negative predictions. An example - * with prediction score greater than or equal to this threshold is identified as an positive, - * and negative otherwise. The default value is 0.5. + * Sets the threshold that separates positive predictions from negative predictions + * in Binary Logistic Regression. An example with prediction score greater than or equal to + * this threshold is identified as an positive, and negative otherwise. The default value is 0.5. */ @Experimental def setThreshold(threshold: Double): this.type = { @@ -61,20 +72,68 @@ class LogisticRegressionModel ( override protected def predictPoint(dataMatrix: Vector, weightMatrix: Vector, intercept: Double) = { - val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept - val score = 1.0 / (1.0 + math.exp(-margin)) - threshold match { - case Some(t) => if (score > t) 1.0 else 0.0 - case None => score + require(dataMatrix.size == numFeatures) + + // If dataMatrix and weightMatrix have the same dimension, it's binary logistic regression. + if (numClasses == 2) { + require(numFeatures == weightMatrix.size) + val margin = dot(weights, dataMatrix) + intercept + val score = 1.0 / (1.0 + math.exp(-margin)) + threshold match { + case Some(t) => if (score > t) 1.0 else 0.0 + case None => score + } + } else { + val dataWithBiasSize = weightMatrix.size / (numClasses - 1) + + val weightsArray = weights match { + case dv: DenseVector => dv.values + case _ => + throw new IllegalArgumentException( + s"weights only supports dense vector but got type ${weights.getClass}.") + } + + val margins = (0 until numClasses - 1).map { i => + var margin = 0.0 + dataMatrix.foreachActive { (index, value) => + if (value != 0.0) margin += value * weightsArray((i * dataWithBiasSize) + index) + } + // Intercept is required to be added into margin. + if (dataMatrix.size + 1 == dataWithBiasSize) { + margin += weightsArray((i * dataWithBiasSize) + dataMatrix.size) + } + margin + } + + /** + * Find the one with maximum margins. If the maxMargin is negative, then the prediction + * result will be the first class. + * + * PS, if you want to compute the probabilities for each outcome instead of the outcome + * with maximum probability, remember to subtract the maxMargin from margins if maxMargin + * is positive to prevent overflow. + */ + var bestClass = 0 + var maxMargin = 0.0 + var i = 0 + while(i < margins.size) { + if (margins(i) > maxMargin) { + maxMargin = margins(i) + bestClass = i + 1 + } + i += 1 + } + bestClass.toDouble } } } /** - * Train a classification model for Logistic Regression using Stochastic Gradient Descent. By - * default L2 regularization is used, which can be changed via - * [[LogisticRegressionWithSGD.optimizer]]. - * NOTE: Labels used in Logistic Regression should be {0, 1}. + * Train a classification model for Binary Logistic Regression + * using Stochastic Gradient Descent. By default L2 regularization is used, + * which can be changed via [[LogisticRegressionWithSGD.optimizer]]. + * NOTE: Labels used in Logistic Regression should be {0, 1, ..., k - 1} + * for k classes multi-label classification problem. * Using [[LogisticRegressionWithLBFGS]] is recommended over this. */ class LogisticRegressionWithSGD private ( @@ -194,9 +253,10 @@ object LogisticRegressionWithSGD { } /** - * Train a classification model for Logistic Regression using Limited-memory BFGS. - * Standard feature scaling and L2 regularization are used by default. - * NOTE: Labels used in Logistic Regression should be {0, 1} + * Train a classification model for Multinomial/Binary Logistic Regression using + * Limited-memory BFGS. Standard feature scaling and L2 regularization are used by default. + * NOTE: Labels used in Logistic Regression should be {0, 1, ..., k - 1} + * for k classes multi-label classification problem. */ class LogisticRegressionWithLBFGS extends GeneralizedLinearAlgorithm[LogisticRegressionModel] with Serializable { @@ -205,9 +265,33 @@ class LogisticRegressionWithLBFGS override val optimizer = new LBFGS(new LogisticGradient, new SquaredL2Updater) - override protected val validators = List(DataValidators.binaryLabelValidator) + override protected val validators = List(multiLabelValidator) + + private def multiLabelValidator: RDD[LabeledPoint] => Boolean = { data => + if (numOfLinearPredictor > 1) { + DataValidators.multiLabelValidator(numOfLinearPredictor + 1)(data) + } else { + DataValidators.binaryLabelValidator(data) + } + } + + /** + * :: Experimental :: + * Set the number of possible outcomes for k classes classification problem in + * Multinomial Logistic Regression. + * By default, it is binary logistic regression so k will be set to 2. + */ + @Experimental + def setNumClasses(numClasses: Int): this.type = { + require(numClasses > 1) + numOfLinearPredictor = numClasses - 1 + if (numClasses > 2) { + optimizer.setGradient(new LogisticGradient(numClasses)) + } + this + } override protected def createModel(weights: Vector, intercept: Double) = { - new LogisticRegressionModel(weights, intercept) + new LogisticRegressionModel(weights, intercept, numFeatures, numOfLinearPredictor + 1) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala index fcb9a3643cc48..9b5c155b0a805 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala @@ -43,15 +43,19 @@ class PowerIterationClusteringModel( * * @param k Number of clusters. * @param maxIterations Maximum number of iterations of the PIC algorithm. + * @param initMode Initialization mode. */ class PowerIterationClustering private[clustering] ( private var k: Int, - private var maxIterations: Int) extends Serializable { + private var maxIterations: Int, + private var initMode: String) extends Serializable { import org.apache.spark.mllib.clustering.PowerIterationClustering._ - /** Constructs a PIC instance with default parameters: {k: 2, maxIterations: 100}. */ - def this() = this(k = 2, maxIterations = 100) + /** Constructs a PIC instance with default parameters: {k: 2, maxIterations: 100, + * initMode: "random"}. + */ + def this() = this(k = 2, maxIterations = 100, initMode = "random") /** * Set the number of clusters. @@ -69,6 +73,18 @@ class PowerIterationClustering private[clustering] ( this } + /** + * Set the initialization mode. This can be either "random" to use a random vector + * as vertex properties, or "degree" to use normalized sum similarities. Default: random. + */ + def setInitializationMode(mode: String): this.type = { + this.initMode = mode match { + case "random" | "degree" => mode + case _ => throw new IllegalArgumentException("Invalid initialization mode: " + mode) + } + this + } + /** * Run the PIC algorithm. * @@ -82,7 +98,10 @@ class PowerIterationClustering private[clustering] ( */ def run(similarities: RDD[(Long, Long, Double)]): PowerIterationClusteringModel = { val w = normalize(similarities) - val w0 = randomInit(w) + val w0 = initMode match { + case "random" => randomInit(w) + case "degree" => initDegreeVector(w) + } pic(w0) } @@ -148,6 +167,20 @@ private[clustering] object PowerIterationClustering extends Logging { GraphImpl.fromExistingRDDs(VertexRDD(v0), g.edges) } + /** + * Generates the degree vector as the vertex properties (v0) to start power iteration. + * It is not exactly the node degrees but just the normalized sum similarities. Call it + * as degree vector because it is used in the PIC paper. + * + * @param g a graph representing the normalized affinity matrix (W) + * @return a graph with edges representing W and vertices representing the degree vector + */ + def initDegreeVector(g: Graph[Double, Double]): Graph[Double, Double] = { + val sum = g.vertices.values.sum() + val v0 = g.vertices.mapValues(_ / sum) + GraphImpl.fromExistingRDDs(VertexRDD(v0), g.edges) + } + /** * Runs power iteration. * @param g input graph with edges representing the normalized affinity matrix (W) and vertices diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala new file mode 100644 index 0000000000000..c6057c7f837b1 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.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.mllib.feature + +import scala.collection.mutable.ArrayBuilder + +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.stat.Statistics +import org.apache.spark.rdd.RDD + +/** + * :: Experimental :: + * Chi Squared selector model. + * + * @param selectedFeatures list of indices to select (filter). Must be ordered asc + */ +@Experimental +class ChiSqSelectorModel (val selectedFeatures: Array[Int]) extends VectorTransformer { + + require(isSorted(selectedFeatures), "Array has to be sorted asc") + + protected def isSorted(array: Array[Int]): Boolean = { + var i = 1 + while (i < array.length) { + if (array(i) < array(i-1)) return false + i += 1 + } + true + } + + /** + * Applies transformation on a vector. + * + * @param vector vector to be transformed. + * @return transformed vector. + */ + override def transform(vector: Vector): Vector = { + compress(vector, selectedFeatures) + } + + /** + * Returns a vector with features filtered. + * Preserves the order of filtered features the same as their indices are stored. + * Might be moved to Vector as .slice + * @param features vector + * @param filterIndices indices of features to filter, must be ordered asc + */ + private def compress(features: Vector, filterIndices: Array[Int]): Vector = { + features match { + case SparseVector(size, indices, values) => + val newSize = filterIndices.length + val newValues = new ArrayBuilder.ofDouble + val newIndices = new ArrayBuilder.ofInt + var i = 0 + var j = 0 + var indicesIdx = 0 + var filterIndicesIdx = 0 + while (i < indices.length && j < filterIndices.length) { + indicesIdx = indices(i) + filterIndicesIdx = filterIndices(j) + if (indicesIdx == filterIndicesIdx) { + newIndices += j + newValues += values(i) + j += 1 + i += 1 + } else { + if (indicesIdx > filterIndicesIdx) { + j += 1 + } else { + i += 1 + } + } + } + // TODO: Sparse representation might be ineffective if (newSize ~= newValues.size) + Vectors.sparse(newSize, newIndices.result(), newValues.result()) + case DenseVector(values) => + val values = features.toArray + Vectors.dense(filterIndices.map(i => values(i))) + case other => + throw new UnsupportedOperationException( + s"Only sparse and dense vectors are supported but got ${other.getClass}.") + } + } +} + +/** + * :: Experimental :: + * Creates a ChiSquared feature selector. + * @param numTopFeatures number of features that selector will select + * (ordered by statistic value descending) + */ +@Experimental +class ChiSqSelector (val numTopFeatures: Int) { + + /** + * Returns a ChiSquared feature selector. + * + * @param data an `RDD[LabeledPoint]` containing the labeled dataset with categorical features. + * Real-valued features will be treated as categorical for each distinct value. + * Apply feature discretizer before using this function. + */ + def fit(data: RDD[LabeledPoint]): ChiSqSelectorModel = { + val indices = Statistics.chiSqTest(data) + .zipWithIndex.sortBy { case (res, _) => -res.statistic } + .take(numTopFeatures) + .map { case (_, indices) => indices } + .sorted + new ChiSqSelectorModel(indices) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala new file mode 100644 index 0000000000000..9591c7966e06a --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala @@ -0,0 +1,162 @@ +/* + * 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.mllib.fpm + +import java.{util => ju} + +import scala.collection.mutable + +import org.apache.spark.{SparkException, HashPartitioner, Logging, Partitioner} +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel + +class FPGrowthModel(val freqItemsets: RDD[(Array[String], Long)]) extends Serializable + +/** + * This class implements Parallel FP-growth algorithm to do frequent pattern matching on input data. + * Parallel FPGrowth (PFP) partitions computation in such a way that each machine executes an + * independent group of mining tasks. More detail of this algorithm can be found at + * [[http://dx.doi.org/10.1145/1454008.1454027, PFP]], and the original FP-growth paper can be + * found at [[http://dx.doi.org/10.1145/335191.335372, FP-growth]] + * + * @param minSupport the minimal support level of the frequent pattern, any pattern appears + * more than (minSupport * size-of-the-dataset) times will be output + * @param numPartitions number of partitions used by parallel FP-growth + */ +class FPGrowth private ( + private var minSupport: Double, + private var numPartitions: Int) extends Logging with Serializable { + + /** + * Constructs a FPGrowth instance with default parameters: + * {minSupport: 0.3, numPartitions: auto} + */ + def this() = this(0.3, -1) + + /** + * Sets the minimal support level (default: 0.3). + */ + def setMinSupport(minSupport: Double): this.type = { + this.minSupport = minSupport + this + } + + /** + * Sets the number of partitions used by parallel FP-growth (default: same as input data). + */ + def setNumPartitions(numPartitions: Int): this.type = { + this.numPartitions = numPartitions + this + } + + /** + * Computes an FP-Growth model that contains frequent itemsets. + * @param data input data set, each element contains a transaction + * @return an [[FPGrowthModel]] + */ + def run(data: RDD[Array[String]]): FPGrowthModel = { + if (data.getStorageLevel == StorageLevel.NONE) { + logWarning("Input data is not cached.") + } + val count = data.count() + val minCount = math.ceil(minSupport * count).toLong + val numParts = if (numPartitions > 0) numPartitions else data.partitions.length + val partitioner = new HashPartitioner(numParts) + val freqItems = genFreqItems(data, minCount, partitioner) + val freqItemsets = genFreqItemsets(data, minCount, freqItems, partitioner) + new FPGrowthModel(freqItemsets) + } + + /** + * Generates frequent items by filtering the input data using minimal support level. + * @param minCount minimum count for frequent itemsets + * @param partitioner partitioner used to distribute items + * @return array of frequent pattern ordered by their frequencies + */ + private def genFreqItems( + data: RDD[Array[String]], + minCount: Long, + partitioner: Partitioner): Array[String] = { + data.flatMap { t => + val uniq = t.toSet + if (t.length != uniq.size) { + throw new SparkException(s"Items in a transaction must be unique but got ${t.toSeq}.") + } + t + }.map(v => (v, 1L)) + .reduceByKey(partitioner, _ + _) + .filter(_._2 >= minCount) + .collect() + .sortBy(-_._2) + .map(_._1) + } + + /** + * Generate frequent itemsets by building FP-Trees, the extraction is done on each partition. + * @param data transactions + * @param minCount minimum count for frequent itemsets + * @param freqItems frequent items + * @param partitioner partitioner used to distribute transactions + * @return an RDD of (frequent itemset, count) + */ + private def genFreqItemsets( + data: RDD[Array[String]], + minCount: Long, + freqItems: Array[String], + partitioner: Partitioner): RDD[(Array[String], Long)] = { + val itemToRank = freqItems.zipWithIndex.toMap + data.flatMap { transaction => + genCondTransactions(transaction, itemToRank, partitioner) + }.aggregateByKey(new FPTree[Int], partitioner.numPartitions)( + (tree, transaction) => tree.add(transaction, 1L), + (tree1, tree2) => tree1.merge(tree2)) + .flatMap { case (part, tree) => + tree.extract(minCount, x => partitioner.getPartition(x) == part) + }.map { case (ranks, count) => + (ranks.map(i => freqItems(i)).toArray, count) + } + } + + /** + * Generates conditional transactions. + * @param transaction a transaction + * @param itemToRank map from item to their rank + * @param partitioner partitioner used to distribute transactions + * @return a map of (target partition, conditional transaction) + */ + private def genCondTransactions( + transaction: Array[String], + itemToRank: Map[String, Int], + partitioner: Partitioner): mutable.Map[Int, Array[Int]] = { + val output = mutable.Map.empty[Int, Array[Int]] + // Filter the basket by frequent items pattern and sort their ranks. + val filtered = transaction.flatMap(itemToRank.get) + ju.Arrays.sort(filtered) + val n = filtered.length + var i = n - 1 + while (i >= 0) { + val item = filtered(i) + val part = partitioner.getPartition(item) + if (!output.contains(part)) { + output(part) = filtered.slice(0, i + 1) + } + i -= 1 + } + output + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala new file mode 100644 index 0000000000000..1d2d777c00793 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala @@ -0,0 +1,134 @@ +/* + * 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.mllib.fpm + +import scala.collection.mutable +import scala.collection.mutable.ListBuffer + +/** + * FP-Tree data structure used in FP-Growth. + * @tparam T item type + */ +private[fpm] class FPTree[T] extends Serializable { + + import FPTree._ + + val root: Node[T] = new Node(null) + + private val summaries: mutable.Map[T, Summary[T]] = mutable.Map.empty + + /** Adds a transaction with count. */ + def add(t: Iterable[T], count: Long = 1L): this.type = { + require(count > 0) + var curr = root + curr.count += count + t.foreach { item => + val summary = summaries.getOrElseUpdate(item, new Summary) + summary.count += count + val child = curr.children.getOrElseUpdate(item, { + val newNode = new Node(curr) + newNode.item = item + summary.nodes += newNode + newNode + }) + child.count += count + curr = child + } + this + } + + /** Merges another FP-Tree. */ + def merge(other: FPTree[T]): this.type = { + other.transactions.foreach { case (t, c) => + add(t, c) + } + this + } + + /** Gets a subtree with the suffix. */ + private def project(suffix: T): FPTree[T] = { + val tree = new FPTree[T] + if (summaries.contains(suffix)) { + val summary = summaries(suffix) + summary.nodes.foreach { node => + var t = List.empty[T] + var curr = node.parent + while (!curr.isRoot) { + t = curr.item :: t + curr = curr.parent + } + tree.add(t, node.count) + } + } + tree + } + + /** Returns all transactions in an iterator. */ + def transactions: Iterator[(List[T], Long)] = getTransactions(root) + + /** Returns all transactions under this node. */ + private def getTransactions(node: Node[T]): Iterator[(List[T], Long)] = { + var count = node.count + node.children.iterator.flatMap { case (item, child) => + getTransactions(child).map { case (t, c) => + count -= c + (item :: t, c) + } + } ++ { + if (count > 0) { + Iterator.single((Nil, count)) + } else { + Iterator.empty + } + } + } + + /** Extracts all patterns with valid suffix and minimum count. */ + def extract( + minCount: Long, + validateSuffix: T => Boolean = _ => true): Iterator[(List[T], Long)] = { + summaries.iterator.flatMap { case (item, summary) => + if (validateSuffix(item) && summary.count >= minCount) { + Iterator.single((item :: Nil, summary.count)) ++ + project(item).extract(minCount).map { case (t, c) => + (item :: t, c) + } + } else { + Iterator.empty + } + } + } +} + +private[fpm] object FPTree { + + /** Representing a node in an FP-Tree. */ + class Node[T](val parent: Node[T]) extends Serializable { + var item: T = _ + var count: Long = 0L + val children: mutable.Map[T, Node[T]] = mutable.Map.empty + + def isRoot: Boolean = parent == null + } + + /** Summary of a item in an FP-Tree. */ + private class Summary[T] extends Serializable { + var count: Long = 0L + val nodes: ListBuffer[Node[T]] = ListBuffer.empty + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index 1ca0f36c6ac34..0acdab797e8f3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.optimization import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.{DenseVector, Vector, Vectors} import org.apache.spark.mllib.linalg.BLAS.{axpy, dot, scal} import org.apache.spark.mllib.util.MLUtils @@ -55,24 +55,86 @@ abstract class Gradient extends Serializable { /** * :: DeveloperApi :: - * Compute gradient and loss for a logistic loss function, as used in binary classification. - * See also the documentation for the precise formulation. + * Compute gradient and loss for a multinomial logistic loss function, as used + * in multi-class classification (it is also used in binary logistic regression). + * + * In `The Elements of Statistical Learning: Data Mining, Inference, and Prediction, 2nd Edition` + * by Trevor Hastie, Robert Tibshirani, and Jerome Friedman, which can be downloaded from + * http://statweb.stanford.edu/~tibs/ElemStatLearn/ , Eq. (4.17) on page 119 gives the formula of + * multinomial logistic regression model. A simple calculation shows that + * + * P(y=0|x, w) = 1 / (1 + \sum_i^{K-1} \exp(x w_i)) + * P(y=1|x, w) = exp(x w_1) / (1 + \sum_i^{K-1} \exp(x w_i)) + * ... + * P(y=K-1|x, w) = exp(x w_{K-1}) / (1 + \sum_i^{K-1} \exp(x w_i)) + * + * for K classes multiclass classification problem. + * + * The model weights w = (w_1, w_2, ..., w_{K-1})^T becomes a matrix which has dimension of + * (K-1) * (N+1) if the intercepts are added. If the intercepts are not added, the dimension + * will be (K-1) * N. + * + * As a result, the loss of objective function for a single instance of data can be written as + * l(w, x) = -log P(y|x, w) = -\alpha(y) log P(y=0|x, w) - (1-\alpha(y)) log P(y|x, w) + * = log(1 + \sum_i^{K-1}\exp(x w_i)) - (1-\alpha(y)) x w_{y-1} + * = log(1 + \sum_i^{K-1}\exp(margins_i)) - (1-\alpha(y)) margins_{y-1} + * + * where \alpha(i) = 1 if i != 0, and + * \alpha(i) = 0 if i == 0, + * margins_i = x w_i. + * + * For optimization, we have to calculate the first derivative of the loss function, and + * a simple calculation shows that + * + * \frac{\partial l(w, x)}{\partial w_{ij}} + * = (\exp(x w_i) / (1 + \sum_k^{K-1} \exp(x w_k)) - (1-\alpha(y)\delta_{y, i+1})) * x_j + * = multiplier_i * x_j + * + * where \delta_{i, j} = 1 if i == j, + * \delta_{i, j} = 0 if i != j, and + * multiplier + * = \exp(margins_i) / (1 + \sum_k^{K-1} \exp(margins_i)) - (1-\alpha(y)\delta_{y, i+1}) + * + * If any of margins is larger than 709.78, the numerical computation of multiplier and loss + * function will be suffered from arithmetic overflow. This issue occurs when there are outliers + * in data which are far away from hyperplane, and this will cause the failing of training once + * infinity / infinity is introduced. Note that this is only a concern when max(margins) > 0. + * + * Fortunately, when max(margins) = maxMargin > 0, the loss function and the multiplier can be + * easily rewritten into the following equivalent numerically stable formula. + * + * l(w, x) = log(1 + \sum_i^{K-1}\exp(margins_i)) - (1-\alpha(y)) margins_{y-1} + * = log(\exp(-maxMargin) + \sum_i^{K-1}\exp(margins_i - maxMargin)) + maxMargin + * - (1-\alpha(y)) margins_{y-1} + * = log(1 + sum) + maxMargin - (1-\alpha(y)) margins_{y-1} + * + * where sum = \exp(-maxMargin) + \sum_i^{K-1}\exp(margins_i - maxMargin) - 1. + * + * Note that each term, (margins_i - maxMargin) in \exp is smaller than zero; as a result, + * overflow will not happen with this formula. + * + * For multiplier, similar trick can be applied as the following, + * + * multiplier = \exp(margins_i) / (1 + \sum_k^{K-1} \exp(margins_i)) - (1-\alpha(y)\delta_{y, i+1}) + * = \exp(margins_i - maxMargin) / (1 + sum) - (1-\alpha(y)\delta_{y, i+1}) + * + * where each term in \exp is also smaller than zero, so overflow is not a concern. + * + * For the detailed mathematical derivation, see the reference at + * http://www.slideshare.net/dbtsai/2014-0620-mlor-36132297 + * + * @param numClasses the number of possible outcomes for k classes classification problem in + * Multinomial Logistic Regression. By default, it is binary logistic regression + * so numClasses will be set to 2. */ @DeveloperApi -class LogisticGradient extends Gradient { - override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { - val margin = -1.0 * dot(data, weights) - val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label - val gradient = data.copy - scal(gradientMultiplier, gradient) - val loss = - if (label > 0) { - // The following is equivalent to log(1 + exp(margin)) but more numerically stable. - MLUtils.log1pExp(margin) - } else { - MLUtils.log1pExp(margin) - margin - } +class LogisticGradient(numClasses: Int) extends Gradient { + def this() = this(2) + + override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { + val gradient = Vectors.zeros(weights.size) + val loss = compute(data, label, weights, gradient) (gradient, loss) } @@ -81,14 +143,104 @@ class LogisticGradient extends Gradient { label: Double, weights: Vector, cumGradient: Vector): Double = { - val margin = -1.0 * dot(data, weights) - val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label - axpy(gradientMultiplier, data, cumGradient) - if (label > 0) { - // The following is equivalent to log(1 + exp(margin)) but more numerically stable. - MLUtils.log1pExp(margin) - } else { - MLUtils.log1pExp(margin) - margin + val dataSize = data.size + + // (weights.size / dataSize + 1) is number of classes + require(weights.size % dataSize == 0 && numClasses == weights.size / dataSize + 1) + numClasses match { + case 2 => + /** + * For Binary Logistic Regression. + * + * Although the loss and gradient calculation for multinomial one is more generalized, + * and multinomial one can also be used in binary case, we still implement a specialized + * binary version for performance reason. + */ + val margin = -1.0 * dot(data, weights) + val multiplier = (1.0 / (1.0 + math.exp(margin))) - label + axpy(multiplier, data, cumGradient) + if (label > 0) { + // The following is equivalent to log(1 + exp(margin)) but more numerically stable. + MLUtils.log1pExp(margin) + } else { + MLUtils.log1pExp(margin) - margin + } + case _ => + /** + * For Multinomial Logistic Regression. + */ + val weightsArray = weights match { + case dv: DenseVector => dv.values + case _ => + throw new IllegalArgumentException( + s"weights only supports dense vector but got type ${weights.getClass}.") + } + val cumGradientArray = cumGradient match { + case dv: DenseVector => dv.values + case _ => + throw new IllegalArgumentException( + s"cumGradient only supports dense vector but got type ${cumGradient.getClass}.") + } + + // marginY is margins(label - 1) in the formula. + var marginY = 0.0 + var maxMargin = Double.NegativeInfinity + var maxMarginIndex = 0 + + val margins = Array.tabulate(numClasses - 1) { i => + var margin = 0.0 + data.foreachActive { (index, value) => + if (value != 0.0) margin += value * weightsArray((i * dataSize) + index) + } + if (i == label.toInt - 1) marginY = margin + if (margin > maxMargin) { + maxMargin = margin + maxMarginIndex = i + } + margin + } + + /** + * When maxMargin > 0, the original formula will cause overflow as we discuss + * in the previous comment. + * We address this by subtracting maxMargin from all the margins, so it's guaranteed + * that all of the new margins will be smaller than zero to prevent arithmetic overflow. + */ + val sum = { + var temp = 0.0 + if (maxMargin > 0) { + for (i <- 0 until numClasses - 1) { + margins(i) -= maxMargin + if (i == maxMarginIndex) { + temp += math.exp(-maxMargin) + } else { + temp += math.exp(margins(i)) + } + } + } else { + for (i <- 0 until numClasses - 1) { + temp += math.exp(margins(i)) + } + } + temp + } + + for (i <- 0 until numClasses - 1) { + val multiplier = math.exp(margins(i)) / (sum + 1.0) - { + if (label != 0.0 && label == i + 1) 1.0 else 0.0 + } + data.foreachActive { (index, value) => + if (value != 0.0) cumGradientArray(i * dataSize + index) += multiplier * value + } + } + + val loss = if (label > 0.0) math.log1p(sum) - marginY else math.log1p(sum) + + if (maxMargin > 0) { + loss + maxMargin + } else { + loss + } } } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala index fef062e02b6ec..ccd93b318bc23 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala @@ -19,13 +19,11 @@ package org.apache.spark.mllib.optimization import org.jblas.{DoubleMatrix, SimpleBlas} -import org.apache.spark.annotation.DeveloperApi - /** * Object used to solve nonnegative least squares problems using a modified * projected gradient method. */ -private[mllib] object NNLS { +private[spark] object NNLS { class Workspace(val n: Int) { val scratch = new DoubleMatrix(n, 1) val grad = new DoubleMatrix(n, 1) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 5f84677be238d..a5ffe888ca880 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -615,7 +615,7 @@ class ALS private ( * Given A^T A and A^T b, find the x minimising ||Ax - b||_2, possibly subject * to nonnegativity constraints if `nonnegative` is true. */ - def solveLeastSquares(ata: DoubleMatrix, atb: DoubleMatrix, + private def solveLeastSquares(ata: DoubleMatrix, atb: DoubleMatrix, ws: NNLS.Workspace): Array[Double] = { if (!nonnegative) { Solve.solvePositive(ata, atb).data diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 0287f04e2c777..17de215b97f9d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -98,6 +98,23 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] protected var validateData: Boolean = true + /** + * In `GeneralizedLinearModel`, only single linear predictor is allowed for both weights + * and intercept. However, for multinomial logistic regression, with K possible outcomes, + * we are training K-1 independent binary logistic regression models which requires K-1 sets + * of linear predictor. + * + * As a result, the workaround here is if more than two sets of linear predictors are needed, + * we construct bigger `weights` vector which can hold both weights and intercepts. + * If the intercepts are added, the dimension of `weights` will be + * (numOfLinearPredictor) * (numFeatures + 1) . If the intercepts are not added, + * the dimension of `weights` will be (numOfLinearPredictor) * numFeatures. + * + * Thus, the intercepts will be encapsulated into weights, and we leave the value of intercept + * in GeneralizedLinearModel as zero. + */ + protected var numOfLinearPredictor: Int = 1 + /** * Whether to perform feature scaling before model training to reduce the condition numbers * which can significantly help the optimizer converging faster. The scaling correction will be @@ -106,6 +123,11 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] */ private var useFeatureScaling = false + /** + * The dimension of training features. + */ + protected var numFeatures: Int = 0 + /** * Set if the algorithm should use feature scaling to improve the convergence during optimization. */ @@ -141,8 +163,28 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] * RDD of LabeledPoint entries. */ def run(input: RDD[LabeledPoint]): M = { - val numFeatures: Int = input.first().features.size - val initialWeights = Vectors.dense(new Array[Double](numFeatures)) + numFeatures = input.first().features.size + + /** + * When `numOfLinearPredictor > 1`, the intercepts are encapsulated into weights, + * so the `weights` will include the intercepts. When `numOfLinearPredictor == 1`, + * the intercept will be stored as separated value in `GeneralizedLinearModel`. + * This will result in different behaviors since when `numOfLinearPredictor == 1`, + * users have no way to set the initial intercept, while in the other case, users + * can set the intercepts as part of weights. + * + * TODO: See if we can deprecate `intercept` in `GeneralizedLinearModel`, and always + * have the intercept as part of weights to have consistent design. + */ + val initialWeights = { + if (numOfLinearPredictor == 1) { + Vectors.dense(new Array[Double](numFeatures)) + } else if (addIntercept) { + Vectors.dense(new Array[Double]((numFeatures + 1) * numOfLinearPredictor)) + } else { + Vectors.dense(new Array[Double](numFeatures * numOfLinearPredictor)) + } + } run(input, initialWeights) } @@ -151,6 +193,7 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] * of LabeledPoint entries starting from the initial weights provided. */ def run(input: RDD[LabeledPoint], initialWeights: Vector): M = { + numFeatures = input.first().features.size if (input.getStorageLevel == StorageLevel.NONE) { logWarning("The input data is not directly cached, which may hurt performance if its" @@ -182,14 +225,14 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] * Currently, it's only enabled in LogisticRegressionWithLBFGS */ val scaler = if (useFeatureScaling) { - (new StandardScaler).fit(input.map(x => x.features)) + (new StandardScaler(withStd = true, withMean = false)).fit(input.map(x => x.features)) } else { null } // Prepend an extra variable consisting of all 1.0's for the intercept. val data = if (addIntercept) { - if(useFeatureScaling) { + if (useFeatureScaling) { input.map(labeledPoint => (labeledPoint.label, appendBias(scaler.transform(labeledPoint.features)))) } else { @@ -203,21 +246,31 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] } } - val initialWeightsWithIntercept = if (addIntercept) { + /** + * TODO: For better convergence, in logistic regression, the intercepts should be computed + * from the prior probability distribution of the outcomes; for linear regression, + * the intercept should be set as the average of response. + */ + val initialWeightsWithIntercept = if (addIntercept && numOfLinearPredictor == 1) { appendBias(initialWeights) } else { + /** If `numOfLinearPredictor > 1`, initialWeights already contains intercepts. */ initialWeights } val weightsWithIntercept = optimizer.optimize(data, initialWeightsWithIntercept) - val intercept = if (addIntercept) weightsWithIntercept(weightsWithIntercept.size - 1) else 0.0 - var weights = - if (addIntercept) { - Vectors.dense(weightsWithIntercept.toArray.slice(0, weightsWithIntercept.size - 1)) - } else { - weightsWithIntercept - } + val intercept = if (addIntercept && numOfLinearPredictor == 1) { + weightsWithIntercept(weightsWithIntercept.size - 1) + } else { + 0.0 + } + + var weights = if (addIntercept && numOfLinearPredictor == 1) { + Vectors.dense(weightsWithIntercept.toArray.slice(0, weightsWithIntercept.size - 1)) + } else { + weightsWithIntercept + } /** * The weights and intercept are trained in the scaled space; we're converting them back to @@ -228,7 +281,29 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] * is the coefficient in the original space, and v_i is the variance of the column i. */ if (useFeatureScaling) { - weights = scaler.transform(weights) + if (numOfLinearPredictor == 1) { + weights = scaler.transform(weights) + } else { + /** + * For `numOfLinearPredictor > 1`, we have to transform the weights back to the original + * scale for each set of linear predictor. Note that the intercepts have to be explicitly + * excluded when `addIntercept == true` since the intercepts are part of weights now. + */ + var i = 0 + val n = weights.size / numOfLinearPredictor + val weightsArray = weights.toArray + while (i < numOfLinearPredictor) { + val start = i * n + val end = (i + 1) * n - { if (addIntercept) 1 else 0 } + + val partialWeightsArray = scaler.transform( + Vectors.dense(weightsArray.slice(start, end))).toArray + + System.arraycopy(partialWeightsArray, 0, weightsArray, start, partialWeightsArray.size) + i += 1 + } + weights = Vectors.dense(weightsArray) + } } // Warn at the end of the run as well, for increased visibility. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala index 45f95482a1def..be335a1aca58a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala @@ -34,11 +34,27 @@ object DataValidators extends Logging { * * @return True if labels are all zero or one, false otherwise. */ - val binaryLabelValidator: RDD[LabeledPoint] => Boolean = { data => + val binaryLabelValidator: RDD[LabeledPoint] => Boolean = { data => val numInvalid = data.filter(x => x.label != 1.0 && x.label != 0.0).count() if (numInvalid != 0) { logError("Classification labels should be 0 or 1. Found " + numInvalid + " invalid labels") } numInvalid == 0 } + + /** + * Function to check if labels used for k class multi-label classification are + * in the range of {0, 1, ..., k - 1}. + * + * @return True if labels are all in the range of {0, 1, ..., k-1}, false otherwise. + */ + def multiLabelValidator(k: Int): RDD[LabeledPoint] => Boolean = { data => + val numInvalid = data.filter(x => + x.label - x.label.toInt != 0.0 || x.label < 0 || x.label > k - 1).count() + if (numInvalid != 0) { + logError("Classification labels should be in {0 to " + (k - 1) + "}. " + + "Found " + numInvalid + " invalid labels") + } + numInvalid == 0 + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala index 07aff56fb7d2f..ee08c3c32760e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala @@ -444,4 +444,15 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext with Logging { val (strUserFactors, _) = ALS.train(strRatings, rank = 2, maxIter = 4) assert(strUserFactors.first()._1.getClass === classOf[String]) } + + test("nonnegative constraint") { + val (ratings, _) = genImplicitTestData(numUsers = 20, numItems = 40, rank = 2, noiseStd = 0.01) + val (userFactors, itemFactors) = ALS.train(ratings, rank = 2, maxIter = 4, nonnegative = true) + def isNonnegative(factors: RDD[(Int, Array[Float])]): Boolean = { + factors.values.map { _.forall(_ >= 0.0) }.reduce(_ && _) + } + assert(isNonnegative(userFactors)) + assert(isNonnegative(itemFactors)) + // TODO: Validate the solution. + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index 94b0e00f37267..3fb45938f75db 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -17,13 +17,14 @@ package org.apache.spark.mllib.classification +import scala.util.control.Breaks._ import scala.util.Random import scala.collection.JavaConversions._ import org.scalatest.FunSuite import org.scalatest.Matchers -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.util.{LocalClusterSparkContext, MLlibTestSparkContext} import org.apache.spark.mllib.util.TestingUtils._ @@ -55,6 +56,97 @@ object LogisticRegressionSuite { val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Vectors.dense(Array(x1(i))))) testData } + + /** + * Generates `k` classes multinomial synthetic logistic input in `n` dimensional space given the + * model weights and mean/variance of the features. The synthetic data will be drawn from + * the probability distribution constructed by weights using the following formula. + * + * P(y = 0 | x) = 1 / norm + * P(y = 1 | x) = exp(x * w_1) / norm + * P(y = 2 | x) = exp(x * w_2) / norm + * ... + * P(y = k-1 | x) = exp(x * w_{k-1}) / norm + * where norm = 1 + exp(x * w_1) + exp(x * w_2) + ... + exp(x * w_{k-1}) + * + * @param weights matrix is flatten into a vector; as a result, the dimension of weights vector + * will be (k - 1) * (n + 1) if `addIntercept == true`, and + * if `addIntercept != true`, the dimension will be (k - 1) * n. + * @param xMean the mean of the generated features. Lots of time, if the features are not properly + * standardized, the algorithm with poor implementation will have difficulty + * to converge. + * @param xVariance the variance of the generated features. + * @param addIntercept whether to add intercept. + * @param nPoints the number of instance of generated data. + * @param seed the seed for random generator. For consistent testing result, it will be fixed. + */ + def generateMultinomialLogisticInput( + weights: Array[Double], + xMean: Array[Double], + xVariance: Array[Double], + addIntercept: Boolean, + nPoints: Int, + seed: Int): Seq[LabeledPoint] = { + val rnd = new Random(seed) + + val xDim = xMean.size + val xWithInterceptsDim = if (addIntercept) xDim + 1 else xDim + val nClasses = weights.size / xWithInterceptsDim + 1 + + val x = Array.fill[Vector](nPoints)(Vectors.dense(Array.fill[Double](xDim)(rnd.nextGaussian()))) + + x.map(vector => { + // This doesn't work if `vector` is a sparse vector. + val vectorArray = vector.toArray + var i = 0 + while (i < vectorArray.size) { + vectorArray(i) = vectorArray(i) * math.sqrt(xVariance(i)) + xMean(i) + i += 1 + } + }) + + val y = (0 until nPoints).map { idx => + val xArray = x(idx).toArray + val margins = Array.ofDim[Double](nClasses) + val probs = Array.ofDim[Double](nClasses) + + for (i <- 0 until nClasses - 1) { + for (j <- 0 until xDim) margins(i + 1) += weights(i * xWithInterceptsDim + j) * xArray(j) + if (addIntercept) margins(i + 1) += weights((i + 1) * xWithInterceptsDim - 1) + } + // Preventing the overflow when we compute the probability + val maxMargin = margins.max + if (maxMargin > 0) for (i <-0 until nClasses) margins(i) -= maxMargin + + // Computing the probabilities for each class from the margins. + val norm = { + var temp = 0.0 + for (i <- 0 until nClasses) { + probs(i) = math.exp(margins(i)) + temp += probs(i) + } + temp + } + for (i <-0 until nClasses) probs(i) /= norm + + // Compute the cumulative probability so we can generate a random number and assign a label. + for (i <- 1 until nClasses) probs(i) += probs(i - 1) + val p = rnd.nextDouble() + var y = 0 + breakable { + for (i <- 0 until nClasses) { + if (p < probs(i)) { + y = i + break + } + } + } + y + } + + val testData = (0 until nPoints).map(i => LabeledPoint(y(i), x(i))) + testData + } } class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext with Matchers { @@ -285,6 +377,91 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext with M assert(modelB1.weights(0) !~== modelB3.weights(0) * 1.0E6 absTol 0.1) } + test("multinomial logistic regression with LBFGS") { + val nPoints = 10000 + + /** + * The following weights and xMean/xVariance are computed from iris dataset with lambda = 0.2. + * As a result, we are actually drawing samples from probability distribution of built model. + */ + val weights = Array( + -0.57997, 0.912083, -0.371077, -0.819866, 2.688191, + -0.16624, -0.84355, -0.048509, -0.301789, 4.170682) + + val xMean = Array(5.843, 3.057, 3.758, 1.199) + val xVariance = Array(0.6856, 0.1899, 3.116, 0.581) + + val testData = LogisticRegressionSuite.generateMultinomialLogisticInput( + weights, xMean, xVariance, true, nPoints, 42) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + val lr = new LogisticRegressionWithLBFGS().setIntercept(true).setNumClasses(3) + lr.optimizer.setConvergenceTol(1E-15).setNumIterations(200) + + val model = lr.run(testRDD) + + /** + * The following is the instruction to reproduce the model using R's glmnet package. + * + * First of all, using the following scala code to save the data into `path`. + * + * testRDD.map(x => x.label+ ", " + x.features(0) + ", " + x.features(1) + ", " + + * x.features(2) + ", " + x.features(3)).saveAsTextFile("path") + * + * Using the following R code to load the data and train the model using glmnet package. + * + * library("glmnet") + * data <- read.csv("path", header=FALSE) + * label = factor(data$V1) + * features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) + * weights = coef(glmnet(features,label, family="multinomial", alpha = 0, lambda = 0)) + * + * The model weights of mutinomial logstic regression in R have `K` set of linear predictors + * for `K` classes classification problem; however, only `K-1` set is required if the first + * outcome is chosen as a "pivot", and the other `K-1` outcomes are separately regressed against + * the pivot outcome. This can be done by subtracting the first weights from those `K-1` set + * weights. The mathematical discussion and proof can be found here: + * http://en.wikipedia.org/wiki/Multinomial_logistic_regression + * + * weights1 = weights$`1` - weights$`0` + * weights2 = weights$`2` - weights$`0` + * + * > weights1 + * 5 x 1 sparse Matrix of class "dgCMatrix" + * s0 + * 2.6228269 + * data.V2 -0.5837166 + * data.V3 0.9285260 + * data.V4 -0.3783612 + * data.V5 -0.8123411 + * > weights2 + * 5 x 1 sparse Matrix of class "dgCMatrix" + * s0 + * 4.11197445 + * data.V2 -0.16918650 + * data.V3 -0.81104784 + * data.V4 -0.06463799 + * data.V5 -0.29198337 + */ + + val weightsR = Vectors.dense(Array( + -0.5837166, 0.9285260, -0.3783612, -0.8123411, 2.6228269, + -0.1691865, -0.811048, -0.0646380, -0.2919834, 4.1119745)) + + assert(model.weights ~== weightsR relTol 0.05) + + val validationData = LogisticRegressionSuite.generateMultinomialLogisticInput( + weights, xMean, xVariance, true, nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) + // The validation accuracy is not good since this model (even the original weights) doesn't have + // very steep curve in logistic function so that when we draw samples from distribution, it's + // very easy to assign to another labels. However, this prediction result is consistent to R. + validatePrediction(model.predict(validationRDD.map(_.features)).collect(), validationData, 0.47) + + } + } class LogisticRegressionClusterSuite extends FunSuite with LocalClusterSparkContext { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala index 2bae465d392aa..03ecd9ca730be 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala @@ -55,6 +55,16 @@ class PowerIterationClusteringSuite extends FunSuite with MLlibTestSparkContext predictions(c) += i } assert(predictions.toSet == Set((0 to 3).toSet, (4 to 15).toSet)) + + val model2 = new PowerIterationClustering() + .setK(2) + .setInitializationMode("degree") + .run(sc.parallelize(similarities, 2)) + val predictions2 = Array.fill(2)(mutable.Set.empty[Long]) + model2.assignments.collect().foreach { case (i, c) => + predictions2(c) += i + } + assert(predictions2.toSet == Set((0 to 3).toSet, (4 to 15).toSet)) } test("normalize and powerIter") { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/ChiSqSelectorSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/ChiSqSelectorSuite.scala new file mode 100644 index 0000000000000..747f5914598ec --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/ChiSqSelectorSuite.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.mllib.feature + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.util.MLlibTestSparkContext + +class ChiSqSelectorSuite extends FunSuite with MLlibTestSparkContext { + + /* + * Contingency tables + * feature0 = {8.0, 0.0} + * class 0 1 2 + * 8.0||1|0|1| + * 0.0||0|2|0| + * + * feature1 = {7.0, 9.0} + * class 0 1 2 + * 7.0||1|0|0| + * 9.0||0|2|1| + * + * feature2 = {0.0, 6.0, 8.0, 5.0} + * class 0 1 2 + * 0.0||1|0|0| + * 6.0||0|1|0| + * 8.0||0|1|0| + * 5.0||0|0|1| + * + * Use chi-squared calculator from Internet + */ + + test("ChiSqSelector transform test (sparse & dense vector)") { + val labeledDiscreteData = sc.parallelize( + Seq(LabeledPoint(0.0, Vectors.sparse(3, Array((0, 8.0), (1, 7.0)))), + LabeledPoint(1.0, Vectors.sparse(3, Array((1, 9.0), (2, 6.0)))), + LabeledPoint(1.0, Vectors.dense(Array(0.0, 9.0, 8.0))), + LabeledPoint(2.0, Vectors.dense(Array(8.0, 9.0, 5.0)))), 2) + val preFilteredData = + Set(LabeledPoint(0.0, Vectors.dense(Array(0.0))), + LabeledPoint(1.0, Vectors.dense(Array(6.0))), + LabeledPoint(1.0, Vectors.dense(Array(8.0))), + LabeledPoint(2.0, Vectors.dense(Array(5.0)))) + val model = new ChiSqSelector(1).fit(labeledDiscreteData) + val filteredData = labeledDiscreteData.map { lp => + LabeledPoint(lp.label, model.transform(lp.features)) + }.collect().toSet + assert(filteredData == preFilteredData) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala new file mode 100644 index 0000000000000..71ef60da6dd32 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala @@ -0,0 +1,73 @@ +/* + * 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.mllib.fpm + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.util.MLlibTestSparkContext + +class FPGrowthSuite extends FunSuite with MLlibTestSparkContext { + + test("FP-Growth") { + val transactions = Seq( + "r z h k p", + "z y x w v u t s", + "s x o n r", + "x z y m t s q e", + "z", + "x z y r q t p") + .map(_.split(" ")) + val rdd = sc.parallelize(transactions, 2).cache() + + val fpg = new FPGrowth() + + val model6 = fpg + .setMinSupport(0.9) + .setNumPartitions(1) + .run(rdd) + assert(model6.freqItemsets.count() === 0) + + val model3 = fpg + .setMinSupport(0.5) + .setNumPartitions(2) + .run(rdd) + val freqItemsets3 = model3.freqItemsets.collect().map { case (items, count) => + (items.toSet, count) + } + val expected = Set( + (Set("s"), 3L), (Set("z"), 5L), (Set("x"), 4L), (Set("t"), 3L), (Set("y"), 3L), + (Set("r"), 3L), + (Set("x", "z"), 3L), (Set("t", "y"), 3L), (Set("t", "x"), 3L), (Set("s", "x"), 3L), + (Set("y", "x"), 3L), (Set("y", "z"), 3L), (Set("t", "z"), 3L), + (Set("y", "x", "z"), 3L), (Set("t", "x", "z"), 3L), (Set("t", "y", "z"), 3L), + (Set("t", "y", "x"), 3L), + (Set("t", "y", "x", "z"), 3L)) + assert(freqItemsets3.toSet === expected) + + val model2 = fpg + .setMinSupport(0.3) + .setNumPartitions(4) + .run(rdd) + assert(model2.freqItemsets.count() === 54) + + val model1 = fpg + .setMinSupport(0.1) + .setNumPartitions(8) + .run(rdd) + assert(model1.freqItemsets.count() === 625) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPTreeSuite.scala new file mode 100644 index 0000000000000..04017f67c311d --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPTreeSuite.scala @@ -0,0 +1,115 @@ +/* + * 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.mllib.fpm + +import scala.language.existentials + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.util.MLlibTestSparkContext + +class FPTreeSuite extends FunSuite with MLlibTestSparkContext { + + test("add transaction") { + val tree = new FPTree[String] + .add(Seq("a", "b", "c")) + .add(Seq("a", "b", "y")) + .add(Seq("b")) + + assert(tree.root.children.size == 2) + assert(tree.root.children.contains("a")) + assert(tree.root.children("a").item.equals("a")) + assert(tree.root.children("a").count == 2) + assert(tree.root.children.contains("b")) + assert(tree.root.children("b").item.equals("b")) + assert(tree.root.children("b").count == 1) + var child = tree.root.children("a") + assert(child.children.size == 1) + assert(child.children.contains("b")) + assert(child.children("b").item.equals("b")) + assert(child.children("b").count == 2) + child = child.children("b") + assert(child.children.size == 2) + assert(child.children.contains("c")) + assert(child.children.contains("y")) + assert(child.children("c").item.equals("c")) + assert(child.children("y").item.equals("y")) + assert(child.children("c").count == 1) + assert(child.children("y").count == 1) + } + + test("merge tree") { + val tree1 = new FPTree[String] + .add(Seq("a", "b", "c")) + .add(Seq("a", "b", "y")) + .add(Seq("b")) + + val tree2 = new FPTree[String] + .add(Seq("a", "b")) + .add(Seq("a", "b", "c")) + .add(Seq("a", "b", "c", "d")) + .add(Seq("a", "x")) + .add(Seq("a", "x", "y")) + .add(Seq("c", "n")) + .add(Seq("c", "m")) + + val tree3 = tree1.merge(tree2) + + assert(tree3.root.children.size == 3) + assert(tree3.root.children("a").count == 7) + assert(tree3.root.children("b").count == 1) + assert(tree3.root.children("c").count == 2) + val child1 = tree3.root.children("a") + assert(child1.children.size == 2) + assert(child1.children("b").count == 5) + assert(child1.children("x").count == 2) + val child2 = child1.children("b") + assert(child2.children.size == 2) + assert(child2.children("y").count == 1) + assert(child2.children("c").count == 3) + val child3 = child2.children("c") + assert(child3.children.size == 1) + assert(child3.children("d").count == 1) + val child4 = child1.children("x") + assert(child4.children.size == 1) + assert(child4.children("y").count == 1) + val child5 = tree3.root.children("c") + assert(child5.children.size == 2) + assert(child5.children("n").count == 1) + assert(child5.children("m").count == 1) + } + + test("extract freq itemsets") { + val tree = new FPTree[String] + .add(Seq("a", "b", "c")) + .add(Seq("a", "b", "y")) + .add(Seq("a", "b")) + .add(Seq("a")) + .add(Seq("b")) + .add(Seq("b", "n")) + + val freqItemsets = tree.extract(3L).map { case (items, count) => + (items.toSet, count) + }.toSet + val expected = Set( + (Set("a"), 4L), + (Set("b"), 5L), + (Set("a", "b"), 3L)) + assert(freqItemsets === expected) + } +} diff --git a/network/common/pom.xml b/network/common/pom.xml index 5a9bbe105d9f1..8f7c924d6b3a3 100644 --- a/network/common/pom.xml +++ b/network/common/pom.xml @@ -101,18 +101,6 @@ - - org.apache.maven.plugins - maven-shade-plugin - - false - - - com.google.guava:guava - - - - diff --git a/pom.xml b/pom.xml index b855f2371b7f0..e25eced877578 100644 --- a/pom.xml +++ b/pom.xml @@ -337,25 +337,51 @@ + + + + org.eclipse.jetty + jetty-http + ${jetty.version} + provided + + + org.eclipse.jetty + jetty-continuation + ${jetty.version} + provided + + + org.eclipse.jetty + jetty-servlet + ${jetty.version} + provided + org.eclipse.jetty jetty-util ${jetty.version} + provided org.eclipse.jetty jetty-security ${jetty.version} + provided org.eclipse.jetty jetty-plus ${jetty.version} + provided org.eclipse.jetty jetty-server ${jetty.version} + provided com.google.guava @@ -363,6 +389,8 @@ 14.0.1 provided + + org.apache.commons commons-lang3 @@ -1276,10 +1304,28 @@ false + org.spark-project.spark:unused + + org.eclipse.jetty:jetty-io + org.eclipse.jetty:jetty-http + org.eclipse.jetty:jetty-continuation + org.eclipse.jetty:jetty-servlet + org.eclipse.jetty:jetty-plus + org.eclipse.jetty:jetty-security + org.eclipse.jetty:jetty-util + org.eclipse.jetty:jetty-server + com.google.guava:guava + + org.eclipse.jetty + org.spark-project.jetty + + org.eclipse.jetty.** + + com.google.common org.spark-project.guava @@ -1597,6 +1643,7 @@ external/kafka + external/kafka-assembly diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 14ba03ed4634b..78de1f0652741 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -66,6 +66,10 @@ object MimaExcludes { "org.apache.spark.mllib.linalg.Matrix.isTransposed"), ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.mllib.linalg.Matrix.foreachActive") + ) ++ Seq( + // SPARK-5540 + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.recommendation.ALS.solveLeastSquares") ) ++ Seq( // SPARK-3325 ProblemFilters.exclude[MissingMethodProblem]( @@ -127,6 +131,12 @@ object MimaExcludes { // SPARK-5315 Spark Streaming Java API returns Scala DStream ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.streaming.api.java.JavaDStreamLike.reduceByWindow") + ) ++ Seq( + // SPARK-5461 Graph should have isCheckpointed, getCheckpointFiles methods + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.graphx.Graph.getCheckpointFiles"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.graphx.Graph.isCheckpointed") ) case v if v.startsWith("1.2") => diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index ded4b5443a904..fbc8983b953b7 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -44,8 +44,9 @@ object BuildCommons { sparkKinesisAsl) = Seq("yarn", "yarn-stable", "java8-tests", "ganglia-lgpl", "kinesis-asl").map(ProjectRef(buildLocation, _)) - val assemblyProjects@Seq(assembly, examples, networkYarn) = - Seq("assembly", "examples", "network-yarn").map(ProjectRef(buildLocation, _)) + val assemblyProjects@Seq(assembly, examples, networkYarn, streamingKafkaAssembly) = + Seq("assembly", "examples", "network-yarn", "streaming-kafka-assembly") + .map(ProjectRef(buildLocation, _)) val tools = ProjectRef(buildLocation, "tools") // Root project. @@ -300,7 +301,14 @@ object Assembly { sys.props.get("hadoop.version") .getOrElse(SbtPomKeys.effectivePom.value.getProperties.get("hadoop.version").asInstanceOf[String]) }, - jarName in assembly := s"${moduleName.value}-${version.value}-hadoop${hadoopVersion.value}.jar", + jarName in assembly <<= (version, moduleName, hadoopVersion) map { (v, mName, hv) => + if (mName.contains("streaming-kafka-assembly")) { + // This must match the same name used in maven (see external/kafka-assembly/pom.xml) + s"${mName}-${v}.jar" + } else { + s"${mName}-${v}-hadoop${hv}.jar" + } + }, mergeStrategy in assembly := { case PathList("org", "datanucleus", xs @ _*) => MergeStrategy.discard case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard diff --git a/python/pyspark/context.py b/python/pyspark/context.py index c0dec16ac1b25..bf1f61c8504ed 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -192,7 +192,8 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, # Create a temporary directory inside spark.local.dir: local_dir = self._jvm.org.apache.spark.util.Utils.getLocalDir(self._jsc.sc().conf()) self._temp_dir = \ - self._jvm.org.apache.spark.util.Utils.createTempDir(local_dir).getAbsolutePath() + self._jvm.org.apache.spark.util.Utils.createTempDir(local_dir, "pyspark") \ + .getAbsolutePath() # profiling stats collected for each PythonRDD if self._conf.get("spark.python.profile", "false") == "true": diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index b8bda835174b2..0ffb41d02f6f6 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -70,6 +70,7 @@ class SpecialLengths(object): PYTHON_EXCEPTION_THROWN = -2 TIMING_DATA = -3 END_OF_STREAM = -4 + NULL = -5 class Serializer(object): @@ -133,6 +134,8 @@ def load_stream(self, stream): def _write_with_length(self, obj, stream): serialized = self.dumps(obj) + if serialized is None: + raise ValueError("serialized value should not be None") if len(serialized) > (1 << 31): raise ValueError("can not serialize object larger than 2G") write_int(len(serialized), stream) @@ -145,8 +148,10 @@ def _read_with_length(self, stream): length = read_int(stream) if length == SpecialLengths.END_OF_DATA_SECTION: raise EOFError + elif length == SpecialLengths.NULL: + return None obj = stream.read(length) - if obj == "": + if len(obj) < length: raise EOFError return self.loads(obj) @@ -484,6 +489,8 @@ def loads(self, stream): length = read_int(stream) if length == SpecialLengths.END_OF_DATA_SECTION: raise EOFError + elif length == SpecialLengths.NULL: + return None s = stream.read(length) return s.decode("utf-8") if self.use_unicode else s diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 3f2d7ac82585f..32bff0c7e8c55 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -2124,6 +2124,10 @@ def head(self, n=None): return rs[0] if rs else None return self.take(n) + def first(self): + """ Return the first row. """ + return self.head() + def tail(self): raise NotImplemented @@ -2159,7 +2163,7 @@ def select(self, *cols): else: cols = [c._jc for c in cols] jcols = ListConverter().convert(cols, self._sc._gateway._gateway_client) - jdf = self._jdf.select(self._jdf.toColumnArray(jcols)) + jdf = self._jdf.select(self.sql_ctx._sc._jvm.Dsl.toColumns(jcols)) return DataFrame(jdf, self.sql_ctx) def filter(self, condition): @@ -2189,7 +2193,7 @@ def groupBy(self, *cols): else: cols = [c._jc for c in cols] jcols = ListConverter().convert(cols, self._sc._gateway._gateway_client) - jdf = self._jdf.groupBy(self._jdf.toColumnArray(jcols)) + jdf = self._jdf.groupBy(self.sql_ctx._sc._jvm.Dsl.toColumns(jcols)) return GroupedDataFrame(jdf, self.sql_ctx) def agg(self, *exprs): @@ -2278,14 +2282,17 @@ def agg(self, *exprs): :param exprs: list or aggregate columns or a map from column name to agregate methods. """ + assert exprs, "exprs should not be empty" if len(exprs) == 1 and isinstance(exprs[0], dict): jmap = MapConverter().convert(exprs[0], self.sql_ctx._sc._gateway._gateway_client) jdf = self._jdf.agg(jmap) else: # Columns - assert all(isinstance(c, Column) for c in exprs), "all exprs should be Columns" - jdf = self._jdf.agg(*exprs) + assert all(isinstance(c, Column) for c in exprs), "all exprs should be Column" + jcols = ListConverter().convert([c._jc for c in exprs[1:]], + self.sql_ctx._sc._gateway._gateway_client) + jdf = self._jdf.agg(exprs[0]._jc, self.sql_ctx._sc._jvm.Dsl.toColumns(jcols)) return DataFrame(jdf, self.sql_ctx) @dfapi @@ -2347,7 +2354,7 @@ def _create_column_from_literal(literal): def _create_column_from_name(name): sc = SparkContext._active_spark_context - return sc._jvm.Column(name) + return sc._jvm.IncomputableColumn(name) def _scalaMethod(name): @@ -2371,7 +2378,7 @@ def _(self): return _ -def _bin_op(name, pass_literal_through=False): +def _bin_op(name, pass_literal_through=True): """ Create a method for given binary operator Keyword arguments: @@ -2465,10 +2472,10 @@ def __init__(self, jc, jdf=None, sql_ctx=None): # __getattr__ = _bin_op("getField") # string methods - rlike = _bin_op("rlike", pass_literal_through=True) - like = _bin_op("like", pass_literal_through=True) - startswith = _bin_op("startsWith", pass_literal_through=True) - endswith = _bin_op("endsWith", pass_literal_through=True) + rlike = _bin_op("rlike") + like = _bin_op("like") + startswith = _bin_op("startsWith") + endswith = _bin_op("endsWith") upper = _unary_op("upper") lower = _unary_op("lower") @@ -2476,7 +2483,6 @@ def substr(self, startPos, pos): if type(startPos) != type(pos): raise TypeError("Can not mix the type") if isinstance(startPos, (int, long)): - jc = self._jc.substr(startPos, pos) elif isinstance(startPos, Column): jc = self._jc.substr(startPos._jc, pos._jc) @@ -2507,16 +2513,21 @@ def cast(self, dataType): return Column(self._jc.cast(jdt), self._jdf, self.sql_ctx) +def _to_java_column(col): + if isinstance(col, Column): + jcol = col._jc + else: + jcol = _create_column_from_name(col) + return jcol + + def _aggregate_func(name): """ Create a function for aggregator by name""" def _(col): sc = SparkContext._active_spark_context - if isinstance(col, Column): - jcol = col._jc - else: - jcol = _create_column_from_name(col) - jc = getattr(sc._jvm.org.apache.spark.sql.Dsl, name)(jcol) + jc = getattr(sc._jvm.Dsl, name)(_to_java_column(col)) return Column(jc) + return staticmethod(_) @@ -2524,13 +2535,31 @@ class Aggregator(object): """ A collections of builtin aggregators """ - max = _aggregate_func("max") - min = _aggregate_func("min") - avg = mean = _aggregate_func("mean") - sum = _aggregate_func("sum") - first = _aggregate_func("first") - last = _aggregate_func("last") - count = _aggregate_func("count") + AGGS = [ + 'lit', 'col', 'column', 'upper', 'lower', 'sqrt', 'abs', + 'min', 'max', 'first', 'last', 'count', 'avg', 'mean', 'sum', 'sumDistinct', + ] + for _name in AGGS: + locals()[_name] = _aggregate_func(_name) + del _name + + @staticmethod + def countDistinct(col, *cols): + sc = SparkContext._active_spark_context + jcols = ListConverter().convert([_to_java_column(c) for c in cols], + sc._gateway._gateway_client) + jc = sc._jvm.Dsl.countDistinct(_to_java_column(col), + sc._jvm.Dsl.toColumns(jcols)) + return Column(jc) + + @staticmethod + def approxCountDistinct(col, rsd=None): + sc = SparkContext._active_spark_context + if rsd is None: + jc = sc._jvm.Dsl.approxCountDistinct(_to_java_column(col)) + else: + jc = sc._jvm.Dsl.approxCountDistinct(_to_java_column(col), rsd) + return Column(jc) def _test(): diff --git a/python/pyspark/streaming/kafka.py b/python/pyspark/streaming/kafka.py new file mode 100644 index 0000000000000..19ad71f99d4d5 --- /dev/null +++ b/python/pyspark/streaming/kafka.py @@ -0,0 +1,83 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from py4j.java_collections import MapConverter +from py4j.java_gateway import java_import, Py4JError + +from pyspark.storagelevel import StorageLevel +from pyspark.serializers import PairDeserializer, NoOpSerializer +from pyspark.streaming import DStream + +__all__ = ['KafkaUtils', 'utf8_decoder'] + + +def utf8_decoder(s): + """ Decode the unicode as UTF-8 """ + return s and s.decode('utf-8') + + +class KafkaUtils(object): + + @staticmethod + def createStream(ssc, zkQuorum, groupId, topics, kafkaParams={}, + storageLevel=StorageLevel.MEMORY_AND_DISK_SER_2, + keyDecoder=utf8_decoder, valueDecoder=utf8_decoder): + """ + Create an input stream that pulls messages from a Kafka Broker. + + :param ssc: StreamingContext object + :param zkQuorum: Zookeeper quorum (hostname:port,hostname:port,..). + :param groupId: The group id for this consumer. + :param topics: Dict of (topic_name -> numPartitions) to consume. + Each partition is consumed in its own thread. + :param kafkaParams: Additional params for Kafka + :param storageLevel: RDD storage level. + :param keyDecoder: A function used to decode key (default is utf8_decoder) + :param valueDecoder: A function used to decode value (default is utf8_decoder) + :return: A DStream object + """ + java_import(ssc._jvm, "org.apache.spark.streaming.kafka.KafkaUtils") + + kafkaParams.update({ + "zookeeper.connect": zkQuorum, + "group.id": groupId, + "zookeeper.connection.timeout.ms": "10000", + }) + if not isinstance(topics, dict): + raise TypeError("topics should be dict") + jtopics = MapConverter().convert(topics, ssc.sparkContext._gateway._gateway_client) + jparam = MapConverter().convert(kafkaParams, ssc.sparkContext._gateway._gateway_client) + jlevel = ssc._sc._getJavaStorageLevel(storageLevel) + + def getClassByName(name): + return ssc._jvm.org.apache.spark.util.Utils.classForName(name) + + try: + array = getClassByName("[B") + decoder = getClassByName("kafka.serializer.DefaultDecoder") + jstream = ssc._jvm.KafkaUtils.createStream(ssc._jssc, array, array, decoder, decoder, + jparam, jtopics, jlevel) + except Py4JError, e: + # TODO: use --jar once it also work on driver + if not e.message or 'call a package' in e.message: + print "No kafka package, please put the assembly jar into classpath:" + print " $ bin/spark-submit --driver-class-path external/kafka-assembly/target/" + \ + "scala-*/spark-streaming-kafka-assembly-*.jar" + raise e + ser = PairDeserializer(NoOpSerializer(), NoOpSerializer()) + stream = DStream(jstream, ssc, ser) + return stream.map(lambda (k, v): (keyDecoder(k), valueDecoder(v))) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index bec1961f26393..c7d0622d65f25 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -47,9 +47,10 @@ from pyspark.conf import SparkConf from pyspark.context import SparkContext +from pyspark.rdd import RDD from pyspark.files import SparkFiles from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer, \ - CloudPickleSerializer, CompressedSerializer + CloudPickleSerializer, CompressedSerializer, UTF8Deserializer, NoOpSerializer from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter from pyspark.sql import SQLContext, IntegerType, Row, ArrayType, StructType, StructField, \ UserDefinedType, DoubleType @@ -716,6 +717,13 @@ def test_sample(self): wr_s21 = rdd.sample(True, 0.4, 21).collect() self.assertNotEqual(set(wr_s11), set(wr_s21)) + def test_null_in_rdd(self): + jrdd = self.sc._jvm.PythonUtils.generateRDDWithNull(self.sc._jsc) + rdd = RDD(jrdd, self.sc, UTF8Deserializer()) + self.assertEqual([u"a", None, u"b"], rdd.collect()) + rdd = RDD(jrdd, self.sc, NoOpSerializer()) + self.assertEqual(["a", None, "b"], rdd.collect()) + def test_multiple_python_java_RDD_conversions(self): # Regression test for SPARK-5361 data = [ @@ -1029,9 +1037,11 @@ def test_aggregator(self): g = df.groupBy() self.assertEqual([99, 100], sorted(g.agg({'key': 'max', 'value': 'count'}).collect()[0])) self.assertEqual([Row(**{"AVG(key#0)": 49.5})], g.mean().collect()) - # TODO(davies): fix aggregators + from pyspark.sql import Aggregator as Agg - # self.assertEqual((0, '100'), tuple(g.agg(Agg.first(df.key), Agg.last(df.value)).first())) + self.assertEqual((0, u'99'), tuple(g.agg(Agg.first(df.key), Agg.last(df.value)).first())) + self.assertTrue(95 < g.agg(Agg.approxCountDistinct(df.key)).first()[0]) + self.assertEqual(100, g.agg(Agg.countDistinct(df.value)).first()[0]) def test_help_command(self): # Regression test for SPARK-5464 diff --git a/repl/pom.xml b/repl/pom.xml index ae7c31aef4f5f..bd39b90fd8714 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -87,6 +87,30 @@ scalacheck_${scala.binary.version} test + + + + org.eclipse.jetty + jetty-server + + + org.eclipse.jetty + jetty-plus + + + org.eclipse.jetty + jetty-util + + + org.eclipse.jetty + jetty-http + + + + + org.scala-lang + scala-library + target/scala-${scala.binary.version}/classes diff --git a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala index 5ee325008a5cd..9805609120005 100644 --- a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala @@ -17,13 +17,13 @@ package org.apache.spark.repl -import java.io.{ByteArrayOutputStream, InputStream} +import java.io.{ByteArrayOutputStream, InputStream, FileNotFoundException} import java.net.{URI, URL, URLEncoder} import java.util.concurrent.{Executors, ExecutorService} import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.spark.{SparkConf, SparkEnv} +import org.apache.spark.{SparkConf, SparkEnv, Logging} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.Utils import org.apache.spark.util.ParentClassLoader @@ -37,7 +37,7 @@ import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ * Allows the user to specify if user class path should be first */ class ExecutorClassLoader(conf: SparkConf, classUri: String, parent: ClassLoader, - userClassPathFirst: Boolean) extends ClassLoader { + userClassPathFirst: Boolean) extends ClassLoader with Logging { val uri = new URI(classUri) val directory = uri.getPath @@ -45,7 +45,7 @@ class ExecutorClassLoader(conf: SparkConf, classUri: String, parent: ClassLoader // Hadoop FileSystem object for our URI, if it isn't using HTTP var fileSystem: FileSystem = { - if (uri.getScheme() == "http") { + if (Set("http", "https", "ftp").contains(uri.getScheme)) { null } else { FileSystem.get(uri, SparkHadoopUtil.get.newConfiguration(conf)) @@ -78,20 +78,30 @@ class ExecutorClassLoader(conf: SparkConf, classUri: String, parent: ClassLoader if (fileSystem != null) { fileSystem.open(new Path(directory, pathInDirectory)) } else { - if (SparkEnv.get.securityManager.isAuthenticationEnabled()) { + val url = if (SparkEnv.get.securityManager.isAuthenticationEnabled()) { val uri = new URI(classUri + "/" + urlEncode(pathInDirectory)) val newuri = Utils.constructURIForAuthentication(uri, SparkEnv.get.securityManager) - newuri.toURL().openStream() + newuri.toURL } else { - new URL(classUri + "/" + urlEncode(pathInDirectory)).openStream() + new URL(classUri + "/" + urlEncode(pathInDirectory)) } + + Utils.setupSecureURLConnection(url.openConnection(), SparkEnv.get.securityManager) + .getInputStream } } val bytes = readAndTransformClass(name, inputStream) inputStream.close() Some(defineClass(name, bytes, 0, bytes.length)) } catch { - case e: Exception => None + case e: FileNotFoundException => + // We did not find the class + logDebug(s"Did not load class $name from REPL class server at $uri", e) + None + case e: Exception => + // Something bad happened while checking if the class exists + logError(s"Failed to check existence of class $name on REPL class server at $uri", e) + None } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala index 4460c86ed9026..cfe2c7a39a17c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala @@ -25,9 +25,17 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} * @param input the set of expression that should be passed to the script. * @param script the command that should be executed. * @param output the attributes that are produced by the script. + * @param ioschema the input and output schema applied in the execution of the script. */ case class ScriptTransformation( input: Seq[Expression], script: String, output: Seq[Attribute], - child: LogicalPlan) extends UnaryNode + child: LogicalPlan, + ioschema: ScriptInputOutputSchema) extends UnaryNode + +/** + * A placeholder for implementation specific input and output properties when passing data + * to a script. For example, in Hive this would specify which SerDes to use. + */ +trait ScriptInputOutputSchema diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala index 6ab99aa38877f..defdcb2b706f5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala @@ -822,7 +822,8 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru * have a name matching the given name, `null` will be returned. */ def apply(name: String): StructField = { - nameToField.getOrElse(name, throw new IllegalArgumentException(s"Field $name does not exist.")) + nameToField.getOrElse(name, + throw new IllegalArgumentException(s"""Field "$name" does not exist.""")) } /** diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 3e9ef07df9db6..1a0c77d282307 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -76,6 +76,30 @@ scalacheck_${scala.binary.version} test + + com.h2database + h2 + 1.4.183 + test + + + mysql + mysql-connector-java + 5.1.34 + test + + + org.postgresql + postgresql + 9.3-1102-jdbc41 + test + + + com.spotify + docker-client + 2.7.5 + test + target/scala-${scala.binary.version}/classes diff --git a/sql/core/src/main/java/org/apache/spark/sql/jdbc/JDBCUtils.java b/sql/core/src/main/java/org/apache/spark/sql/jdbc/JDBCUtils.java new file mode 100644 index 0000000000000..aa441b2096f18 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/jdbc/JDBCUtils.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc; + +import org.apache.spark.Partition; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.DataFrame; + +public class JDBCUtils { + /** + * Construct a DataFrame representing the JDBC table at the database + * specified by url with table name table. + */ + public static DataFrame jdbcRDD(SQLContext sql, String url, String table) { + Partition[] parts = new Partition[1]; + parts[0] = new JDBCPartition(null, 0); + return sql.baseRelationToDataFrame( + new JDBCRelation(url, table, parts, sql)); + } + + /** + * Construct a DataFrame representing the JDBC table at the database + * specified by url with table name table partitioned by parts. + * Here, parts is an array of expressions suitable for insertion into a WHERE + * clause; each one defines one partition. + */ + public static DataFrame jdbcRDD(SQLContext sql, String url, String table, String[] parts) { + Partition[] partitions = new Partition[parts.length]; + for (int i = 0; i < parts.length; i++) + partitions[i] = new JDBCPartition(parts[i], i); + return sql.baseRelationToDataFrame( + new JDBCRelation(url, table, partitions, sql)); + } + + private static JavaJDBCTrampoline trampoline = new JavaJDBCTrampoline(); + + public static void createJDBCTable(DataFrame rdd, String url, String table, boolean allowExisting) { + trampoline.createJDBCTable(rdd, url, table, allowExisting); + } + + public static void insertIntoJDBC(DataFrame rdd, String url, String table, boolean overwrite) { + trampoline.insertIntoJDBC(rdd, url, table, overwrite); + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index 174c403059510..6f48d7c3fe1b6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -17,23 +17,26 @@ package org.apache.spark.sql +import scala.annotation.tailrec import scala.language.implicitConversions import org.apache.spark.sql.Dsl.lit -import org.apache.spark.sql.catalyst.analysis.{UnresolvedStar, UnresolvedAttribute} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.{Project, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.{Subquery, Project, LogicalPlan} import org.apache.spark.sql.types._ -object Column { - /** - * Creates a [[Column]] based on the given column name. Same as [[Dsl.col]]. - */ - def apply(colName: String): Column = new Column(colName) +private[sql] object Column { + + def apply(colName: String): Column = new IncomputableColumn(colName) + + def apply(expr: Expression): Column = new IncomputableColumn(expr) + + def apply(sqlContext: SQLContext, plan: LogicalPlan, expr: Expression): Column = { + new ComputableColumn(sqlContext, plan, expr) + } - /** For internal pattern matching. */ - private[sql] def unapply(col: Column): Option[Expression] = Some(col.expr) + def unapply(col: Column): Option[Expression] = Some(col.expr) } @@ -53,44 +56,42 @@ object Column { * */ // TODO: Improve documentation. -class Column( - sqlContext: Option[SQLContext], - plan: Option[LogicalPlan], - protected[sql] val expr: Expression) - extends DataFrame(sqlContext, plan) with ExpressionApi { +trait Column extends DataFrame with ExpressionApi { - /** Turns a Catalyst expression into a `Column`. */ - protected[sql] def this(expr: Expression) = this(None, None, expr) + protected[sql] def expr: Expression /** - * Creates a new `Column` expression based on a column or attribute name. - * The resolution of this is the same as SQL. For example: - * - * - "colName" becomes an expression selecting the column named "colName". - * - "*" becomes an expression selecting all columns. - * - "df.*" becomes an expression selecting all columns in data frame "df". + * Returns true iff the [[Column]] is computable. */ - def this(name: String) = this(name match { - case "*" => UnresolvedStar(None) - case _ if name.endsWith(".*") => UnresolvedStar(Some(name.substring(0, name.length - 2))) - case _ => UnresolvedAttribute(name) - }) + def isComputable: Boolean - override def isComputable: Boolean = sqlContext.isDefined && plan.isDefined + private def constructColumn(other: Column)(newExpr: Expression): Column = { + // Removes all the top level projection and subquery so we can get to the underlying plan. + @tailrec def stripProject(p: LogicalPlan): LogicalPlan = p match { + case Project(_, child) => stripProject(child) + case Subquery(_, child) => stripProject(child) + case _ => p + } - /** - * An implicit conversion function internal to this class. This function creates a new Column - * based on an expression. If the expression itself is not named, it aliases the expression - * by calling it "col". - */ - private[this] implicit def toColumn(expr: Expression): Column = { - val projectedPlan = plan.map { p => - Project(Seq(expr match { + def computableCol(baseCol: ComputableColumn, expr: Expression) = { + val plan = Project(Seq(expr match { case named: NamedExpression => named case unnamed: Expression => Alias(unnamed, "col")() - }), p) + }), baseCol.plan) + Column(baseCol.sqlContext, plan, expr) + } + + (this, other) match { + case (left: ComputableColumn, right: ComputableColumn) => + if (stripProject(left.plan).sameResult(stripProject(right.plan))) { + computableCol(right, newExpr) + } else { + Column(newExpr) + } + case (left: ComputableColumn, _) => computableCol(left, newExpr) + case (_, right: ComputableColumn) => computableCol(right, newExpr) + case (_, _) => Column(newExpr) } - new Column(sqlContext, projectedPlan, expr) } /** @@ -100,7 +101,7 @@ class Column( * df.select( -df("amount") ) * }}} */ - override def unary_- : Column = UnaryMinus(expr) + override def unary_- : Column = constructColumn(null) { UnaryMinus(expr) } /** * Bitwise NOT. @@ -109,7 +110,7 @@ class Column( * df.select( ~df("flags") ) * }}} */ - override def unary_~ : Column = BitwiseNot(expr) + override def unary_~ : Column = constructColumn(null) { BitwiseNot(expr) } /** * Inversion of boolean expression, i.e. NOT. @@ -118,7 +119,7 @@ class Column( * df.select( !df("isActive") ) * }} */ - override def unary_! : Column = Not(expr) + override def unary_! : Column = constructColumn(null) { Not(expr) } /** @@ -129,7 +130,9 @@ class Column( * df.select( df("colA".equalTo(df("colB")) ) * }}} */ - override def === (other: Column): Column = EqualTo(expr, other.expr) + override def === (other: Column): Column = constructColumn(other) { + EqualTo(expr, other.expr) + } /** * Equality test with a literal value. @@ -169,7 +172,9 @@ class Column( * df.select( !(df("colA") === df("colB")) ) * }}} */ - override def !== (other: Column): Column = Not(EqualTo(expr, other.expr)) + override def !== (other: Column): Column = constructColumn(other) { + Not(EqualTo(expr, other.expr)) + } /** * Inequality test with a literal value. @@ -188,7 +193,9 @@ class Column( * people.select( people("age") > Literal(21) ) * }}} */ - override def > (other: Column): Column = GreaterThan(expr, other.expr) + override def > (other: Column): Column = constructColumn(other) { + GreaterThan(expr, other.expr) + } /** * Greater than a literal value. @@ -206,7 +213,9 @@ class Column( * people.select( people("age") < Literal(21) ) * }}} */ - override def < (other: Column): Column = LessThan(expr, other.expr) + override def < (other: Column): Column = constructColumn(other) { + LessThan(expr, other.expr) + } /** * Less than a literal value. @@ -224,7 +233,9 @@ class Column( * people.select( people("age") <= Literal(21) ) * }}} */ - override def <= (other: Column): Column = LessThanOrEqual(expr, other.expr) + override def <= (other: Column): Column = constructColumn(other) { + LessThanOrEqual(expr, other.expr) + } /** * Less than or equal to a literal value. @@ -242,7 +253,9 @@ class Column( * people.select( people("age") >= Literal(21) ) * }}} */ - override def >= (other: Column): Column = GreaterThanOrEqual(expr, other.expr) + override def >= (other: Column): Column = constructColumn(other) { + GreaterThanOrEqual(expr, other.expr) + } /** * Greater than or equal to a literal value. @@ -256,9 +269,11 @@ class Column( /** * Equality test with an expression that is safe for null values. */ - override def <=> (other: Column): Column = other match { - case null => EqualNullSafe(expr, lit(null).expr) - case _ => EqualNullSafe(expr, other.expr) + override def <=> (other: Column): Column = constructColumn(other) { + other match { + case null => EqualNullSafe(expr, lit(null).expr) + case _ => EqualNullSafe(expr, other.expr) + } } /** @@ -269,12 +284,12 @@ class Column( /** * True if the current expression is null. */ - override def isNull: Column = IsNull(expr) + override def isNull: Column = constructColumn(null) { IsNull(expr) } /** * True if the current expression is NOT null. */ - override def isNotNull: Column = IsNotNull(expr) + override def isNotNull: Column = constructColumn(null) { IsNotNull(expr) } /** * Boolean OR with an expression. @@ -283,7 +298,9 @@ class Column( * people.select( people("inSchool") || people("isEmployed") ) * }}} */ - override def || (other: Column): Column = Or(expr, other.expr) + override def || (other: Column): Column = constructColumn(other) { + Or(expr, other.expr) + } /** * Boolean OR with a literal value. @@ -301,7 +318,9 @@ class Column( * people.select( people("inSchool") && people("isEmployed") ) * }}} */ - override def && (other: Column): Column = And(expr, other.expr) + override def && (other: Column): Column = constructColumn(other) { + And(expr, other.expr) + } /** * Boolean AND with a literal value. @@ -315,7 +334,9 @@ class Column( /** * Bitwise AND with an expression. */ - override def & (other: Column): Column = BitwiseAnd(expr, other.expr) + override def & (other: Column): Column = constructColumn(other) { + BitwiseAnd(expr, other.expr) + } /** * Bitwise AND with a literal value. @@ -325,7 +346,9 @@ class Column( /** * Bitwise OR with an expression. */ - override def | (other: Column): Column = BitwiseOr(expr, other.expr) + override def | (other: Column): Column = constructColumn(other) { + BitwiseOr(expr, other.expr) + } /** * Bitwise OR with a literal value. @@ -335,7 +358,9 @@ class Column( /** * Bitwise XOR with an expression. */ - override def ^ (other: Column): Column = BitwiseXor(expr, other.expr) + override def ^ (other: Column): Column = constructColumn(other) { + BitwiseXor(expr, other.expr) + } /** * Bitwise XOR with a literal value. @@ -349,7 +374,9 @@ class Column( * people.select( people("height") + people("weight") ) * }}} */ - override def + (other: Column): Column = Add(expr, other.expr) + override def + (other: Column): Column = constructColumn(other) { + Add(expr, other.expr) + } /** * Sum of this expression and another expression. @@ -367,7 +394,9 @@ class Column( * people.select( people("height") - people("weight") ) * }}} */ - override def - (other: Column): Column = Subtract(expr, other.expr) + override def - (other: Column): Column = constructColumn(other) { + Subtract(expr, other.expr) + } /** * Subtraction. Subtract a literal value from this expression. @@ -385,7 +414,9 @@ class Column( * people.select( people("height") * people("weight") ) * }}} */ - override def * (other: Column): Column = Multiply(expr, other.expr) + override def * (other: Column): Column = constructColumn(other) { + Multiply(expr, other.expr) + } /** * Multiplication this expression and a literal value. @@ -403,7 +434,9 @@ class Column( * people.select( people("height") / people("weight") ) * }}} */ - override def / (other: Column): Column = Divide(expr, other.expr) + override def / (other: Column): Column = constructColumn(other) { + Divide(expr, other.expr) + } /** * Division this expression by a literal value. @@ -417,7 +450,9 @@ class Column( /** * Modulo (a.k.a. remainder) expression. */ - override def % (other: Column): Column = Remainder(expr, other.expr) + override def % (other: Column): Column = constructColumn(other) { + Remainder(expr, other.expr) + } /** * Modulo (a.k.a. remainder) expression. @@ -430,29 +465,40 @@ class Column( * by the evaluated values of the arguments. */ @scala.annotation.varargs - override def in(list: Column*): Column = In(expr, list.map(_.expr)) + override def in(list: Column*): Column = { + new IncomputableColumn(In(expr, list.map(_.expr))) + } - override def like(literal: String): Column = Like(expr, lit(literal).expr) + override def like(literal: String): Column = constructColumn(null) { + Like(expr, lit(literal).expr) + } - override def rlike(literal: String): Column = RLike(expr, lit(literal).expr) + override def rlike(literal: String): Column = constructColumn(null) { + RLike(expr, lit(literal).expr) + } /** * An expression that gets an item at position `ordinal` out of an array. */ - override def getItem(ordinal: Int): Column = GetItem(expr, Literal(ordinal)) + override def getItem(ordinal: Int): Column = constructColumn(null) { + GetItem(expr, Literal(ordinal)) + } /** * An expression that gets a field by name in a [[StructField]]. */ - override def getField(fieldName: String): Column = GetField(expr, fieldName) + override def getField(fieldName: String): Column = constructColumn(null) { + GetField(expr, fieldName) + } /** * An expression that returns a substring. * @param startPos expression for the starting position. * @param len expression for the length of the substring. */ - override def substr(startPos: Column, len: Column): Column = - Substring(expr, startPos.expr, len.expr) + override def substr(startPos: Column, len: Column): Column = { + new IncomputableColumn(Substring(expr, startPos.expr, len.expr)) + } /** * An expression that returns a substring. @@ -461,16 +507,21 @@ class Column( */ override def substr(startPos: Int, len: Int): Column = this.substr(lit(startPos), lit(len)) - override def contains(other: Column): Column = Contains(expr, other.expr) + override def contains(other: Column): Column = constructColumn(other) { + Contains(expr, other.expr) + } override def contains(literal: Any): Column = this.contains(lit(literal)) - - override def startsWith(other: Column): Column = StartsWith(expr, other.expr) + override def startsWith(other: Column): Column = constructColumn(other) { + StartsWith(expr, other.expr) + } override def startsWith(literal: String): Column = this.startsWith(lit(literal)) - override def endsWith(other: Column): Column = EndsWith(expr, other.expr) + override def endsWith(other: Column): Column = constructColumn(other) { + EndsWith(expr, other.expr) + } override def endsWith(literal: String): Column = this.endsWith(lit(literal)) @@ -481,7 +532,7 @@ class Column( * df.select($"colA".as("colB")) * }}} */ - override def as(alias: String): Column = Alias(expr, alias)() + override def as(alias: String): Column = constructColumn(null) { Alias(expr, alias)() } /** * Casts the column to a different data type. @@ -494,7 +545,7 @@ class Column( * df.select(df("colA").cast("int")) * }}} */ - override def cast(to: DataType): Column = Cast(expr, to) + override def cast(to: DataType): Column = constructColumn(null) { Cast(expr, to) } /** * Casts the column to a different data type, using the canonical string representation @@ -505,28 +556,30 @@ class Column( * df.select(df("colA").cast("int")) * }}} */ - override def cast(to: String): Column = Cast(expr, to.toLowerCase match { - case "string" => StringType - case "boolean" => BooleanType - case "byte" => ByteType - case "short" => ShortType - case "int" => IntegerType - case "long" => LongType - case "float" => FloatType - case "double" => DoubleType - case "decimal" => DecimalType.Unlimited - case "date" => DateType - case "timestamp" => TimestampType - case _ => throw new RuntimeException(s"""Unsupported cast type: "$to"""") - }) - - override def desc: Column = SortOrder(expr, Descending) - - override def asc: Column = SortOrder(expr, Ascending) + override def cast(to: String): Column = constructColumn(null) { + Cast(expr, to.toLowerCase match { + case "string" => StringType + case "boolean" => BooleanType + case "byte" => ByteType + case "short" => ShortType + case "int" => IntegerType + case "long" => LongType + case "float" => FloatType + case "double" => DoubleType + case "decimal" => DecimalType.Unlimited + case "date" => DateType + case "timestamp" => TimestampType + case _ => throw new RuntimeException(s"""Unsupported cast type: "$to"""") + }) + } + + override def desc: Column = constructColumn(null) { SortOrder(expr, Descending) } + + override def asc: Column = constructColumn(null) { SortOrder(expr, Ascending) } } -class ColumnName(name: String) extends Column(name) { +class ColumnName(name: String) extends IncomputableColumn(name) { /** Creates a new AttributeReference of type boolean */ def boolean: StructField = StructField(name, BooleanType) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ComputableColumn.scala b/sql/core/src/main/scala/org/apache/spark/sql/ComputableColumn.scala new file mode 100644 index 0000000000000..ac479b26a7c6a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/ComputableColumn.scala @@ -0,0 +1,33 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql + +import scala.language.implicitConversions + +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan + + +private[sql] class ComputableColumn protected[sql]( + sqlContext: SQLContext, + protected[sql] val plan: LogicalPlan, + protected[sql] val expr: Expression) + extends DataFrameImpl(sqlContext, plan) with Column { + + override def isComputable: Boolean = true +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 1096e396591df..385e1ec74f5f7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -19,26 +19,21 @@ package org.apache.spark.sql import java.util.{List => JList} -import scala.language.implicitConversions import scala.reflect.ClassTag -import scala.collection.JavaConversions._ -import com.fasterxml.jackson.core.JsonFactory - -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD -import org.apache.spark.api.python.SerDeUtil import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel -import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.analysis.{ResolvedStar, UnresolvedRelation} -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.{JoinType, Inner} import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.execution.{LogicalRDD, EvaluatePython} -import org.apache.spark.sql.json.JsonRDD -import org.apache.spark.sql.types.{NumericType, StructType} -import org.apache.spark.util.Utils +import org.apache.spark.sql.types.StructType + + +private[sql] object DataFrame { + def apply(sqlContext: SQLContext, logicalPlan: LogicalPlan): DataFrame = { + new DataFrameImpl(sqlContext, logicalPlan) + } +} /** @@ -78,50 +73,14 @@ import org.apache.spark.util.Utils * }}} */ // TODO: Improve documentation. -class DataFrame protected[sql]( - val sqlContext: SQLContext, - private val baseLogicalPlan: LogicalPlan, - operatorsEnabled: Boolean) - extends DataFrameSpecificApi with RDDApi[Row] { - - protected[sql] def this(sqlContext: Option[SQLContext], plan: Option[LogicalPlan]) = - this(sqlContext.orNull, plan.orNull, sqlContext.isDefined && plan.isDefined) - - protected[sql] def this(sqlContext: SQLContext, plan: LogicalPlan) = this(sqlContext, plan, true) - - @transient protected[sql] lazy val queryExecution = sqlContext.executePlan(baseLogicalPlan) - - @transient protected[sql] val logicalPlan: LogicalPlan = baseLogicalPlan match { - // For various commands (like DDL) and queries with side effects, we force query optimization to - // happen right away to let these side effects take place eagerly. - case _: Command | _: InsertIntoTable | _: CreateTableAsSelect[_] |_: WriteToFile => - LogicalRDD(queryExecution.analyzed.output, queryExecution.toRdd)(sqlContext) - case _ => - baseLogicalPlan - } +trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { - /** - * An implicit conversion function internal to this class for us to avoid doing - * "new DataFrame(...)" everywhere. - */ - private implicit def logicalPlanToDataFrame(logicalPlan: LogicalPlan): DataFrame = { - new DataFrame(sqlContext, logicalPlan, true) - } + val sqlContext: SQLContext - /** Returns the list of numeric columns, useful for doing aggregation. */ - protected[sql] def numericColumns: Seq[Expression] = { - schema.fields.filter(_.dataType.isInstanceOf[NumericType]).map { n => - queryExecution.analyzed.resolve(n.name, sqlContext.analyzer.resolver).get - } - } + @DeveloperApi + def queryExecution: SQLContext#QueryExecution - /** Resolves a column name into a Catalyst [[NamedExpression]]. */ - protected[sql] def resolve(colName: String): NamedExpression = { - queryExecution.analyzed.resolve(colName, sqlContext.analyzer.resolver).getOrElse { - throw new RuntimeException( - s"""Cannot resolve column name "$colName" among (${schema.fieldNames.mkString(", ")})""") - } - } + protected[sql] def logicalPlan: LogicalPlan /** Left here for compatibility reasons. */ @deprecated("1.3.0", "use toDataFrame") @@ -142,32 +101,19 @@ class DataFrame protected[sql]( * }}} */ @scala.annotation.varargs - def toDataFrame(colName: String, colNames: String*): DataFrame = { - val newNames = colName +: colNames - require(schema.size == newNames.size, - "The number of columns doesn't match.\n" + - "Old column names: " + schema.fields.map(_.name).mkString(", ") + "\n" + - "New column names: " + newNames.mkString(", ")) - - val newCols = schema.fieldNames.zip(newNames).map { case (oldName, newName) => - apply(oldName).as(newName) - } - select(newCols :_*) - } + def toDataFrame(colName: String, colNames: String*): DataFrame /** Returns the schema of this [[DataFrame]]. */ - override def schema: StructType = queryExecution.analyzed.schema + override def schema: StructType /** Returns all column names and their data types as an array. */ - override def dtypes: Array[(String, String)] = schema.fields.map { field => - (field.name, field.dataType.toString) - } + override def dtypes: Array[(String, String)] /** Returns all column names as an array. */ override def columns: Array[String] = schema.fields.map(_.name) /** Prints the schema to the console in a nice tree format. */ - override def printSchema(): Unit = println(schema.treeString) + override def printSchema(): Unit /** * Cartesian join with another [[DataFrame]]. @@ -176,9 +122,7 @@ class DataFrame protected[sql]( * * @param right Right side of the join operation. */ - override def join(right: DataFrame): DataFrame = { - Join(logicalPlan, right.logicalPlan, joinType = Inner, None) - } + override def join(right: DataFrame): DataFrame /** * Inner join with another [[DataFrame]], using the given join expression. @@ -189,9 +133,7 @@ class DataFrame protected[sql]( * df1.join(df2).where($"df1Key" === $"df2Key") * }}} */ - override def join(right: DataFrame, joinExprs: Column): DataFrame = { - Join(logicalPlan, right.logicalPlan, Inner, Some(joinExprs.expr)) - } + override def join(right: DataFrame, joinExprs: Column): DataFrame /** * Join with another [[DataFrame]], usin g the given join expression. The following performs @@ -205,9 +147,7 @@ class DataFrame protected[sql]( * @param joinExprs Join expression. * @param joinType One of: `inner`, `outer`, `left_outer`, `right_outer`, `semijoin`. */ - override def join(right: DataFrame, joinExprs: Column, joinType: String): DataFrame = { - Join(logicalPlan, right.logicalPlan, JoinType(joinType), Some(joinExprs.expr)) - } + override def join(right: DataFrame, joinExprs: Column, joinType: String): DataFrame /** * Returns a new [[DataFrame]] sorted by the specified column, all in ascending order. @@ -219,9 +159,7 @@ class DataFrame protected[sql]( * }}} */ @scala.annotation.varargs - override def sort(sortCol: String, sortCols: String*): DataFrame = { - orderBy(apply(sortCol), sortCols.map(apply) :_*) - } + override def sort(sortCol: String, sortCols: String*): DataFrame /** * Returns a new [[DataFrame]] sorted by the given expressions. For example: @@ -230,46 +168,26 @@ class DataFrame protected[sql]( * }}} */ @scala.annotation.varargs - override def sort(sortExpr: Column, sortExprs: Column*): DataFrame = { - val sortOrder: Seq[SortOrder] = (sortExpr +: sortExprs).map { col => - col.expr match { - case expr: SortOrder => - expr - case expr: Expression => - SortOrder(expr, Ascending) - } - } - Sort(sortOrder, global = true, logicalPlan) - } + override def sort(sortExpr: Column, sortExprs: Column*): DataFrame /** * Returns a new [[DataFrame]] sorted by the given expressions. * This is an alias of the `sort` function. */ @scala.annotation.varargs - override def orderBy(sortCol: String, sortCols: String*): DataFrame = { - sort(sortCol, sortCols :_*) - } + override def orderBy(sortCol: String, sortCols: String*): DataFrame /** * Returns a new [[DataFrame]] sorted by the given expressions. * This is an alias of the `sort` function. */ @scala.annotation.varargs - override def orderBy(sortExpr: Column, sortExprs: Column*): DataFrame = { - sort(sortExpr, sortExprs :_*) - } + override def orderBy(sortExpr: Column, sortExprs: Column*): DataFrame /** * Selects column based on the column name and return it as a [[Column]]. */ - override def apply(colName: String): Column = colName match { - case "*" => - new Column(ResolvedStar(schema.fieldNames.map(resolve))) - case _ => - val expr = resolve(colName) - new Column(Some(sqlContext), Some(Project(Seq(expr), logicalPlan)), expr) - } + override def apply(colName: String): Column /** * Selects a set of expressions, wrapped in a Product. @@ -279,18 +197,12 @@ class DataFrame protected[sql]( * df.select($"colA", $"colB" + 1) * }}} */ - override def apply(projection: Product): DataFrame = { - require(projection.productArity >= 1) - select(projection.productIterator.map { - case c: Column => c - case o: Any => new Column(Some(sqlContext), None, Literal(o)) - }.toSeq :_*) - } + override def apply(projection: Product): DataFrame /** * Returns a new [[DataFrame]] with an alias set. */ - override def as(name: String): DataFrame = Subquery(name, logicalPlan) + override def as(name: String): DataFrame /** * Selects a set of expressions. @@ -299,15 +211,7 @@ class DataFrame protected[sql]( * }}} */ @scala.annotation.varargs - override def select(cols: Column*): DataFrame = { - val exprs = cols.zipWithIndex.map { - case (Column(expr: NamedExpression), _) => - expr - case (Column(expr: Expression), _) => - Alias(expr, expr.toString)() - } - Project(exprs.toSeq, logicalPlan) - } + override def select(cols: Column*): DataFrame /** * Selects a set of columns. This is a variant of `select` that can only select @@ -320,9 +224,7 @@ class DataFrame protected[sql]( * }}} */ @scala.annotation.varargs - override def select(col: String, cols: String*): DataFrame = { - select((col +: cols).map(new Column(_)) :_*) - } + override def select(col: String, cols: String*): DataFrame /** * Filters rows using the given condition. @@ -333,9 +235,7 @@ class DataFrame protected[sql]( * peopleDf($"age" > 15) * }}} */ - override def filter(condition: Column): DataFrame = { - Filter(condition.expr, logicalPlan) - } + override def filter(condition: Column): DataFrame /** * Filters rows using the given condition. This is an alias for `filter`. @@ -346,7 +246,7 @@ class DataFrame protected[sql]( * peopleDf($"age" > 15) * }}} */ - override def where(condition: Column): DataFrame = filter(condition) + override def where(condition: Column): DataFrame /** * Filters rows using the given condition. This is a shorthand meant for Scala. @@ -357,7 +257,7 @@ class DataFrame protected[sql]( * peopleDf($"age" > 15) * }}} */ - override def apply(condition: Column): DataFrame = filter(condition) + override def apply(condition: Column): DataFrame /** * Groups the [[DataFrame]] using the specified columns, so we can run aggregation on them. @@ -375,9 +275,7 @@ class DataFrame protected[sql]( * }}} */ @scala.annotation.varargs - override def groupBy(cols: Column*): GroupedDataFrame = { - new GroupedDataFrame(this, cols.map(_.expr)) - } + override def groupBy(cols: Column*): GroupedDataFrame /** * Groups the [[DataFrame]] using the specified columns, so we can run aggregation on them. @@ -398,10 +296,7 @@ class DataFrame protected[sql]( * }}} */ @scala.annotation.varargs - override def groupBy(col1: String, cols: String*): GroupedDataFrame = { - val colNames: Seq[String] = col1 +: cols - new GroupedDataFrame(this, colNames.map(colName => resolve(colName))) - } + override def groupBy(col1: String, cols: String*): GroupedDataFrame /** * Aggregates on the entire [[DataFrame]] without groups. @@ -411,7 +306,7 @@ class DataFrame protected[sql]( * df.groupBy().agg(Map("age" -> "max", "salary" -> "avg")) * }} */ - override def agg(exprs: Map[String, String]): DataFrame = groupBy().agg(exprs) + override def agg(exprs: Map[String, String]): DataFrame /** * Aggregates on the entire [[DataFrame]] without groups. @@ -421,7 +316,7 @@ class DataFrame protected[sql]( * df.groupBy().agg(Map("age" -> "max", "salary" -> "avg")) * }} */ - override def agg(exprs: java.util.Map[String, String]): DataFrame = agg(exprs.toMap) + override def agg(exprs: java.util.Map[String, String]): DataFrame /** * Aggregates on the entire [[DataFrame]] without groups. @@ -432,31 +327,31 @@ class DataFrame protected[sql]( * }} */ @scala.annotation.varargs - override def agg(expr: Column, exprs: Column*): DataFrame = groupBy().agg(expr, exprs :_*) + override def agg(expr: Column, exprs: Column*): DataFrame /** * Returns a new [[DataFrame]] by taking the first `n` rows. The difference between this function * and `head` is that `head` returns an array while `limit` returns a new [[DataFrame]]. */ - override def limit(n: Int): DataFrame = Limit(Literal(n), logicalPlan) + override def limit(n: Int): DataFrame /** * Returns a new [[DataFrame]] containing union of rows in this frame and another frame. * This is equivalent to `UNION ALL` in SQL. */ - override def unionAll(other: DataFrame): DataFrame = Union(logicalPlan, other.logicalPlan) + override def unionAll(other: DataFrame): DataFrame /** * Returns a new [[DataFrame]] containing rows only in both this frame and another frame. * This is equivalent to `INTERSECT` in SQL. */ - override def intersect(other: DataFrame): DataFrame = Intersect(logicalPlan, other.logicalPlan) + override def intersect(other: DataFrame): DataFrame /** * Returns a new [[DataFrame]] containing rows in this frame but not in another frame. * This is equivalent to `EXCEPT` in SQL. */ - override def except(other: DataFrame): DataFrame = Except(logicalPlan, other.logicalPlan) + override def except(other: DataFrame): DataFrame /** * Returns a new [[DataFrame]] by sampling a fraction of rows. @@ -465,9 +360,7 @@ class DataFrame protected[sql]( * @param fraction Fraction of rows to generate. * @param seed Seed for sampling. */ - override def sample(withReplacement: Boolean, fraction: Double, seed: Long): DataFrame = { - Sample(fraction, withReplacement, seed, logicalPlan) - } + override def sample(withReplacement: Boolean, fraction: Double, seed: Long): DataFrame /** * Returns a new [[DataFrame]] by sampling a fraction of rows, using a random seed. @@ -475,105 +368,85 @@ class DataFrame protected[sql]( * @param withReplacement Sample with replacement or not. * @param fraction Fraction of rows to generate. */ - override def sample(withReplacement: Boolean, fraction: Double): DataFrame = { - sample(withReplacement, fraction, Utils.random.nextLong) - } + override def sample(withReplacement: Boolean, fraction: Double): DataFrame ///////////////////////////////////////////////////////////////////////////// /** * Returns a new [[DataFrame]] by adding a column. */ - override def addColumn(colName: String, col: Column): DataFrame = { - select(Column("*"), col.as(colName)) - } + override def addColumn(colName: String, col: Column): DataFrame /** * Returns the first `n` rows. */ - override def head(n: Int): Array[Row] = limit(n).collect() + override def head(n: Int): Array[Row] /** * Returns the first row. */ - override def head(): Row = head(1).head + override def head(): Row /** * Returns the first row. Alias for head(). */ - override def first(): Row = head() + override def first(): Row /** * Returns a new RDD by applying a function to all rows of this DataFrame. */ - override def map[R: ClassTag](f: Row => R): RDD[R] = { - rdd.map(f) - } + override def map[R: ClassTag](f: Row => R): RDD[R] /** * Returns a new RDD by first applying a function to all rows of this [[DataFrame]], * and then flattening the results. */ - override def flatMap[R: ClassTag](f: Row => TraversableOnce[R]): RDD[R] = rdd.flatMap(f) + override def flatMap[R: ClassTag](f: Row => TraversableOnce[R]): RDD[R] /** * Returns a new RDD by applying a function to each partition of this DataFrame. */ - override def mapPartitions[R: ClassTag](f: Iterator[Row] => Iterator[R]): RDD[R] = { - rdd.mapPartitions(f) - } - + override def mapPartitions[R: ClassTag](f: Iterator[Row] => Iterator[R]): RDD[R] /** * Applies a function `f` to all rows. */ - override def foreach(f: Row => Unit): Unit = rdd.foreach(f) + override def foreach(f: Row => Unit): Unit /** * Applies a function f to each partition of this [[DataFrame]]. */ - override def foreachPartition(f: Iterator[Row] => Unit): Unit = rdd.foreachPartition(f) + override def foreachPartition(f: Iterator[Row] => Unit): Unit /** * Returns the first `n` rows in the [[DataFrame]]. */ - override def take(n: Int): Array[Row] = head(n) + override def take(n: Int): Array[Row] /** * Returns an array that contains all of [[Row]]s in this [[DataFrame]]. */ - override def collect(): Array[Row] = rdd.collect() + override def collect(): Array[Row] /** * Returns a Java list that contains all of [[Row]]s in this [[DataFrame]]. */ - override def collectAsList(): java.util.List[Row] = java.util.Arrays.asList(rdd.collect() :_*) + override def collectAsList(): java.util.List[Row] /** * Returns the number of rows in the [[DataFrame]]. */ - override def count(): Long = groupBy().count().rdd.collect().head.getLong(0) + override def count(): Long /** * Returns a new [[DataFrame]] that has exactly `numPartitions` partitions. */ - override def repartition(numPartitions: Int): DataFrame = { - sqlContext.applySchema(rdd.repartition(numPartitions), schema) - } + override def repartition(numPartitions: Int): DataFrame - override def persist(): this.type = { - sqlContext.cacheManager.cacheQuery(this) - this - } + override def persist(): this.type - override def persist(newLevel: StorageLevel): this.type = { - sqlContext.cacheManager.cacheQuery(this, None, newLevel) - this - } + override def persist(newLevel: StorageLevel): this.type - override def unpersist(blocking: Boolean): this.type = { - sqlContext.cacheManager.tryUncacheQuery(this, blocking) - this - } + override def unpersist(blocking: Boolean): this.type ///////////////////////////////////////////////////////////////////////////// // I/O @@ -582,10 +455,7 @@ class DataFrame protected[sql]( /** * Returns the content of the [[DataFrame]] as an [[RDD]] of [[Row]]s. */ - override def rdd: RDD[Row] = { - val schema = this.schema - queryExecution.executedPlan.execute().map(ScalaReflection.convertRowToScala(_, schema)) - } + override def rdd: RDD[Row] /** * Registers this RDD as a temporary table using the given name. The lifetime of this temporary @@ -593,18 +463,14 @@ class DataFrame protected[sql]( * * @group schema */ - override def registerTempTable(tableName: String): Unit = { - sqlContext.registerRDDAsTable(this, tableName) - } + override def registerTempTable(tableName: String): Unit /** * Saves the contents of this [[DataFrame]] as a parquet file, preserving the schema. * Files that are written out using this method can be read back in as a [[DataFrame]] * using the `parquetFile` function in [[SQLContext]]. */ - override def saveAsParquetFile(path: String): Unit = { - sqlContext.executePlan(WriteToFile(path, logicalPlan)).toRdd - } + override def saveAsParquetFile(path: String): Unit /** * :: Experimental :: @@ -617,48 +483,26 @@ class DataFrame protected[sql]( * be the target of an `insertInto`. */ @Experimental - override def saveAsTable(tableName: String): Unit = { - sqlContext.executePlan( - CreateTableAsSelect(None, tableName, logicalPlan, allowExisting = false)).toRdd - } + override def saveAsTable(tableName: String): Unit /** * :: Experimental :: * Adds the rows from this RDD to the specified table, optionally overwriting the existing data. */ @Experimental - override def insertInto(tableName: String, overwrite: Boolean): Unit = { - sqlContext.executePlan(InsertIntoTable(UnresolvedRelation(Seq(tableName)), - Map.empty, logicalPlan, overwrite)).toRdd - } + override def insertInto(tableName: String, overwrite: Boolean): Unit /** * Returns the content of the [[DataFrame]] as a RDD of JSON strings. */ - override def toJSON: RDD[String] = { - val rowSchema = this.schema - this.mapPartitions { iter => - val jsonFactory = new JsonFactory() - iter.map(JsonRDD.rowToJSON(rowSchema, jsonFactory)) - } - } + override def toJSON: RDD[String] //////////////////////////////////////////////////////////////////////////// // for Python API //////////////////////////////////////////////////////////////////////////// - /** - * A helpful function for Py4j, convert a list of Column to an array - */ - protected[sql] def toColumnArray(cols: JList[Column]): Array[Column] = { - cols.toList.toArray - } /** * Converts a JavaRDD to a PythonRDD. */ - protected[sql] def javaToPython: JavaRDD[Array[Byte]] = { - val fieldTypes = schema.fields.map(_.dataType) - val jrdd = rdd.map(EvaluatePython.rowToArray(_, fieldTypes)).toJavaRDD() - SerDeUtil.javaToPython(jrdd) - } + protected[sql] def javaToPython: JavaRDD[Array[Byte]] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala new file mode 100644 index 0000000000000..f8fcc25569482 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala @@ -0,0 +1,331 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql + +import java.util.{List => JList} + +import scala.language.implicitConversions +import scala.reflect.ClassTag +import scala.collection.JavaConversions._ + +import com.fasterxml.jackson.core.JsonFactory + +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.api.python.SerDeUtil +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.analysis.{ResolvedStar, UnresolvedRelation} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.{JoinType, Inner} +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.execution.{LogicalRDD, EvaluatePython} +import org.apache.spark.sql.json.JsonRDD +import org.apache.spark.sql.types.{NumericType, StructType} +import org.apache.spark.util.Utils + + +/** + * See [[DataFrame]] for documentation. + */ +private[sql] class DataFrameImpl protected[sql]( + override val sqlContext: SQLContext, + val queryExecution: SQLContext#QueryExecution) + extends DataFrame { + + def this(sqlContext: SQLContext, logicalPlan: LogicalPlan) = { + this(sqlContext, { + val qe = sqlContext.executePlan(logicalPlan) + qe.analyzed // This should force analysis and throw errors if there are any + qe + }) + } + + @transient protected[sql] override val logicalPlan: LogicalPlan = queryExecution.logical match { + // For various commands (like DDL) and queries with side effects, we force query optimization to + // happen right away to let these side effects take place eagerly. + case _: Command | _: InsertIntoTable | _: CreateTableAsSelect[_] |_: WriteToFile => + LogicalRDD(queryExecution.analyzed.output, queryExecution.toRdd)(sqlContext) + case _ => + queryExecution.logical + } + + /** + * An implicit conversion function internal to this class for us to avoid doing + * "new DataFrameImpl(...)" everywhere. + */ + @inline private implicit def logicalPlanToDataFrame(logicalPlan: LogicalPlan): DataFrame = { + new DataFrameImpl(sqlContext, logicalPlan) + } + + protected[sql] def resolve(colName: String): NamedExpression = { + queryExecution.analyzed.resolve(colName, sqlContext.analyzer.resolver).getOrElse { + throw new RuntimeException( + s"""Cannot resolve column name "$colName" among (${schema.fieldNames.mkString(", ")})""") + } + } + + protected[sql] def numericColumns: Seq[Expression] = { + schema.fields.filter(_.dataType.isInstanceOf[NumericType]).map { n => + queryExecution.analyzed.resolve(n.name, sqlContext.analyzer.resolver).get + } + } + + override def toDataFrame(colName: String, colNames: String*): DataFrame = { + val newNames = colName +: colNames + require(schema.size == newNames.size, + "The number of columns doesn't match.\n" + + "Old column names: " + schema.fields.map(_.name).mkString(", ") + "\n" + + "New column names: " + newNames.mkString(", ")) + + val newCols = schema.fieldNames.zip(newNames).map { case (oldName, newName) => + apply(oldName).as(newName) + } + select(newCols :_*) + } + + override def schema: StructType = queryExecution.analyzed.schema + + override def dtypes: Array[(String, String)] = schema.fields.map { field => + (field.name, field.dataType.toString) + } + + override def columns: Array[String] = schema.fields.map(_.name) + + override def printSchema(): Unit = println(schema.treeString) + + override def join(right: DataFrame): DataFrame = { + Join(logicalPlan, right.logicalPlan, joinType = Inner, None) + } + + override def join(right: DataFrame, joinExprs: Column): DataFrame = { + Join(logicalPlan, right.logicalPlan, Inner, Some(joinExprs.expr)) + } + + override def join(right: DataFrame, joinExprs: Column, joinType: String): DataFrame = { + Join(logicalPlan, right.logicalPlan, JoinType(joinType), Some(joinExprs.expr)) + } + + override def sort(sortCol: String, sortCols: String*): DataFrame = { + orderBy(apply(sortCol), sortCols.map(apply) :_*) + } + + override def sort(sortExpr: Column, sortExprs: Column*): DataFrame = { + val sortOrder: Seq[SortOrder] = (sortExpr +: sortExprs).map { col => + col.expr match { + case expr: SortOrder => + expr + case expr: Expression => + SortOrder(expr, Ascending) + } + } + Sort(sortOrder, global = true, logicalPlan) + } + + override def orderBy(sortCol: String, sortCols: String*): DataFrame = { + sort(sortCol, sortCols :_*) + } + + override def orderBy(sortExpr: Column, sortExprs: Column*): DataFrame = { + sort(sortExpr, sortExprs :_*) + } + + override def apply(colName: String): Column = colName match { + case "*" => + Column(ResolvedStar(schema.fieldNames.map(resolve))) + case _ => + val expr = resolve(colName) + Column(sqlContext, Project(Seq(expr), logicalPlan), expr) + } + + override def apply(projection: Product): DataFrame = { + require(projection.productArity >= 1) + select(projection.productIterator.map { + case c: Column => c + case o: Any => Column(Literal(o)) + }.toSeq :_*) + } + + override def as(name: String): DataFrame = Subquery(name, logicalPlan) + + override def select(cols: Column*): DataFrame = { + val exprs = cols.zipWithIndex.map { + case (Column(expr: NamedExpression), _) => + expr + case (Column(expr: Expression), _) => + Alias(expr, expr.toString)() + } + Project(exprs.toSeq, logicalPlan) + } + + override def select(col: String, cols: String*): DataFrame = { + select((col +: cols).map(Column(_)) :_*) + } + + override def filter(condition: Column): DataFrame = { + Filter(condition.expr, logicalPlan) + } + + override def where(condition: Column): DataFrame = { + filter(condition) + } + + override def apply(condition: Column): DataFrame = { + filter(condition) + } + + override def groupBy(cols: Column*): GroupedDataFrame = { + new GroupedDataFrame(this, cols.map(_.expr)) + } + + override def groupBy(col1: String, cols: String*): GroupedDataFrame = { + val colNames: Seq[String] = col1 +: cols + new GroupedDataFrame(this, colNames.map(colName => resolve(colName))) + } + + override def agg(exprs: Map[String, String]): DataFrame = { + groupBy().agg(exprs) + } + + override def agg(exprs: java.util.Map[String, String]): DataFrame = { + agg(exprs.toMap) + } + + override def agg(expr: Column, exprs: Column*): DataFrame = { + groupBy().agg(expr, exprs :_*) + } + + override def limit(n: Int): DataFrame = { + Limit(Literal(n), logicalPlan) + } + + override def unionAll(other: DataFrame): DataFrame = { + Union(logicalPlan, other.logicalPlan) + } + + override def intersect(other: DataFrame): DataFrame = { + Intersect(logicalPlan, other.logicalPlan) + } + + override def except(other: DataFrame): DataFrame = { + Except(logicalPlan, other.logicalPlan) + } + + override def sample(withReplacement: Boolean, fraction: Double, seed: Long): DataFrame = { + Sample(fraction, withReplacement, seed, logicalPlan) + } + + override def sample(withReplacement: Boolean, fraction: Double): DataFrame = { + sample(withReplacement, fraction, Utils.random.nextLong) + } + + ///////////////////////////////////////////////////////////////////////////// + + override def addColumn(colName: String, col: Column): DataFrame = { + select(Column("*"), col.as(colName)) + } + + override def head(n: Int): Array[Row] = limit(n).collect() + + override def head(): Row = head(1).head + + override def first(): Row = head() + + override def map[R: ClassTag](f: Row => R): RDD[R] = rdd.map(f) + + override def flatMap[R: ClassTag](f: Row => TraversableOnce[R]): RDD[R] = rdd.flatMap(f) + + override def mapPartitions[R: ClassTag](f: Iterator[Row] => Iterator[R]): RDD[R] = { + rdd.mapPartitions(f) + } + + override def foreach(f: Row => Unit): Unit = rdd.foreach(f) + + override def foreachPartition(f: Iterator[Row] => Unit): Unit = rdd.foreachPartition(f) + + override def take(n: Int): Array[Row] = head(n) + + override def collect(): Array[Row] = queryExecution.executedPlan.executeCollect() + + override def collectAsList(): java.util.List[Row] = java.util.Arrays.asList(rdd.collect() :_*) + + override def count(): Long = groupBy().count().rdd.collect().head.getLong(0) + + override def repartition(numPartitions: Int): DataFrame = { + sqlContext.applySchema(rdd.repartition(numPartitions), schema) + } + + override def persist(): this.type = { + sqlContext.cacheManager.cacheQuery(this) + this + } + + override def persist(newLevel: StorageLevel): this.type = { + sqlContext.cacheManager.cacheQuery(this, None, newLevel) + this + } + + override def unpersist(blocking: Boolean): this.type = { + sqlContext.cacheManager.tryUncacheQuery(this, blocking) + this + } + + ///////////////////////////////////////////////////////////////////////////// + // I/O + ///////////////////////////////////////////////////////////////////////////// + + override def rdd: RDD[Row] = { + val schema = this.schema + queryExecution.executedPlan.execute().map(ScalaReflection.convertRowToScala(_, schema)) + } + + override def registerTempTable(tableName: String): Unit = { + sqlContext.registerRDDAsTable(this, tableName) + } + + override def saveAsParquetFile(path: String): Unit = { + sqlContext.executePlan(WriteToFile(path, logicalPlan)).toRdd + } + + override def saveAsTable(tableName: String): Unit = { + sqlContext.executePlan( + CreateTableAsSelect(None, tableName, logicalPlan, allowExisting = false)).toRdd + } + + override def insertInto(tableName: String, overwrite: Boolean): Unit = { + sqlContext.executePlan(InsertIntoTable(UnresolvedRelation(Seq(tableName)), + Map.empty, logicalPlan, overwrite)).toRdd + } + + override def toJSON: RDD[String] = { + val rowSchema = this.schema + this.mapPartitions { iter => + val jsonFactory = new JsonFactory() + iter.map(JsonRDD.rowToJSON(rowSchema, jsonFactory)) + } + } + + //////////////////////////////////////////////////////////////////////////// + // for Python API + //////////////////////////////////////////////////////////////////////////// + protected[sql] override def javaToPython: JavaRDD[Array[Byte]] = { + val fieldTypes = schema.fields.map(_.dataType) + val jrdd = rdd.map(EvaluatePython.rowToArray(_, fieldTypes)).toJavaRDD() + SerDeUtil.javaToPython(jrdd) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala index 3499956023d11..b4279a32ffa21 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala @@ -17,8 +17,11 @@ package org.apache.spark.sql +import java.util.{List => JList} + import scala.language.implicitConversions import scala.reflect.runtime.universe.{TypeTag, typeTag} +import scala.collection.JavaConversions._ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.expressions._ @@ -55,17 +58,17 @@ object Dsl { } } - private[this] implicit def toColumn(expr: Expression): Column = new Column(expr) + private[this] implicit def toColumn(expr: Expression): Column = Column(expr) /** * Returns a [[Column]] based on the given column name. */ - def col(colName: String): Column = new Column(colName) + def col(colName: String): Column = Column(colName) /** * Returns a [[Column]] based on the given column name. Alias of [[col]]. */ - def column(colName: String): Column = new Column(colName) + def column(colName: String): Column = Column(colName) /** * Creates a [[Column]] of literal value. @@ -94,7 +97,7 @@ object Dsl { case _ => throw new RuntimeException("Unsupported literal type " + literal.getClass + " " + literal) } - new Column(literalExpr) + Column(literalExpr) } def sum(e: Column): Column = Sum(e.expr) @@ -105,8 +108,7 @@ object Dsl { def countDistinct(expr: Column, exprs: Column*): Column = CountDistinct((expr +: exprs).map(_.expr)) - def approxCountDistinct(e: Column): Column = - ApproxCountDistinct(e.expr) + def approxCountDistinct(e: Column): Column = ApproxCountDistinct(e.expr) def approxCountDistinct(e: Column, rsd: Double): Column = ApproxCountDistinct(e.expr, rsd) @@ -121,6 +123,13 @@ object Dsl { def sqrt(e: Column): Column = Sqrt(e.expr) def abs(e: Column): Column = Abs(e.expr) + /** + * This is a private API for Python + * TODO: move this to a private package + */ + def toColumns(cols: JList[Column]): Seq[Column] = { + cols.toList.toSeq + } // scalastyle:off diff --git a/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataFrame.scala index 1c948cbbfe58f..d3acd41bbf3eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataFrame.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql +import java.util.{List => JList} + import scala.language.implicitConversions import scala.collection.JavaConversions._ @@ -28,7 +30,7 @@ import org.apache.spark.sql.catalyst.plans.logical.Aggregate /** * A set of methods for aggregations on a [[DataFrame]], created by [[DataFrame.groupBy]]. */ -class GroupedDataFrame protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) +class GroupedDataFrame protected[sql](df: DataFrameImpl, groupingExprs: Seq[Expression]) extends GroupedDataFrameApi { private[this] implicit def toDataFrame(aggExprs: Seq[NamedExpression]): DataFrame = { @@ -36,8 +38,8 @@ class GroupedDataFrame protected[sql](df: DataFrame, groupingExprs: Seq[Expressi case expr: NamedExpression => expr case expr: Expression => Alias(expr, expr.toString)() } - new DataFrame(df.sqlContext, - Aggregate(groupingExprs, namedGroupingExprs ++ aggExprs, df.logicalPlan)) + DataFrame( + df.sqlContext, Aggregate(groupingExprs, namedGroupingExprs ++ aggExprs, df.logicalPlan)) } private[this] def aggregateNumericColumns(f: Expression => Expression): Seq[NamedExpression] = { @@ -112,8 +114,7 @@ class GroupedDataFrame protected[sql](df: DataFrame, groupingExprs: Seq[Expressi case expr: NamedExpression => expr case expr: Expression => Alias(expr, expr.toString)() } - - new DataFrame(df.sqlContext, Aggregate(groupingExprs, aggExprs, df.logicalPlan)) + DataFrame(df.sqlContext, Aggregate(groupingExprs, aggExprs, df.logicalPlan)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala b/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala new file mode 100644 index 0000000000000..2f8c695d5654b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala @@ -0,0 +1,160 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql + +import scala.reflect.ClassTag + +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedStar} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.storage.StorageLevel +import org.apache.spark.sql.types.StructType + + +private[sql] class IncomputableColumn(protected[sql] val expr: Expression) extends Column { + + def this(name: String) = this(name match { + case "*" => UnresolvedStar(None) + case _ if name.endsWith(".*") => UnresolvedStar(Some(name.substring(0, name.length - 2))) + case _ => UnresolvedAttribute(name) + }) + + private def err[T](): T = { + throw new UnsupportedOperationException("Cannot run this method on an UncomputableColumn") + } + + override def isComputable: Boolean = false + + override val sqlContext: SQLContext = null + + override def queryExecution = err() + + protected[sql] override def logicalPlan: LogicalPlan = err() + + override def toDataFrame(colName: String, colNames: String*): DataFrame = err() + + override def schema: StructType = err() + + override def dtypes: Array[(String, String)] = err() + + override def columns: Array[String] = err() + + override def printSchema(): Unit = err() + + override def join(right: DataFrame): DataFrame = err() + + override def join(right: DataFrame, joinExprs: Column): DataFrame = err() + + override def join(right: DataFrame, joinExprs: Column, joinType: String): DataFrame = err() + + override def sort(sortCol: String, sortCols: String*): DataFrame = err() + + override def sort(sortExpr: Column, sortExprs: Column*): DataFrame = err() + + override def orderBy(sortCol: String, sortCols: String*): DataFrame = err() + + override def orderBy(sortExpr: Column, sortExprs: Column*): DataFrame = err() + + override def apply(colName: String): Column = err() + + override def apply(projection: Product): DataFrame = err() + + override def select(cols: Column*): DataFrame = err() + + override def select(col: String, cols: String*): DataFrame = err() + + override def filter(condition: Column): DataFrame = err() + + override def where(condition: Column): DataFrame = err() + + override def apply(condition: Column): DataFrame = err() + + override def groupBy(cols: Column*): GroupedDataFrame = err() + + override def groupBy(col1: String, cols: String*): GroupedDataFrame = err() + + override def agg(exprs: Map[String, String]): DataFrame = err() + + override def agg(exprs: java.util.Map[String, String]): DataFrame = err() + + override def agg(expr: Column, exprs: Column*): DataFrame = err() + + override def limit(n: Int): DataFrame = err() + + override def unionAll(other: DataFrame): DataFrame = err() + + override def intersect(other: DataFrame): DataFrame = err() + + override def except(other: DataFrame): DataFrame = err() + + override def sample(withReplacement: Boolean, fraction: Double, seed: Long): DataFrame = err() + + override def sample(withReplacement: Boolean, fraction: Double): DataFrame = err() + + ///////////////////////////////////////////////////////////////////////////// + + override def addColumn(colName: String, col: Column): DataFrame = err() + + override def head(n: Int): Array[Row] = err() + + override def head(): Row = err() + + override def first(): Row = err() + + override def map[R: ClassTag](f: Row => R): RDD[R] = err() + + override def flatMap[R: ClassTag](f: Row => TraversableOnce[R]): RDD[R] = err() + + override def mapPartitions[R: ClassTag](f: Iterator[Row] => Iterator[R]): RDD[R] = err() + + override def foreach(f: Row => Unit): Unit = err() + + override def foreachPartition(f: Iterator[Row] => Unit): Unit = err() + + override def take(n: Int): Array[Row] = err() + + override def collect(): Array[Row] = err() + + override def collectAsList(): java.util.List[Row] = err() + + override def count(): Long = err() + + override def repartition(numPartitions: Int): DataFrame = err() + + override def persist(): this.type = err() + + override def persist(newLevel: StorageLevel): this.type = err() + + override def unpersist(blocking: Boolean): this.type = err() + + override def rdd: RDD[Row] = err() + + override def registerTempTable(tableName: String): Unit = err() + + override def saveAsParquetFile(path: String): Unit = err() + + override def saveAsTable(tableName: String): Unit = err() + + override def insertInto(tableName: String, overwrite: Boolean): Unit = err() + + override def toJSON: RDD[String] = err() + + protected[sql] override def javaToPython: JavaRDD[Array[Byte]] = err() +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 84933dd944837..f4692b3ff59d3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -24,7 +24,7 @@ import scala.collection.immutable import scala.language.implicitConversions import scala.reflect.runtime.universe.TypeTag -import org.apache.spark.SparkContext +import org.apache.spark.{SparkContext, Partition} import org.apache.spark.annotation.{AlphaComponent, DeveloperApi, Experimental} import org.apache.spark.api.java.{JavaSparkContext, JavaRDD} import org.apache.spark.rdd.RDD @@ -36,6 +36,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution._ import org.apache.spark.sql.json._ +import org.apache.spark.sql.jdbc.{JDBCPartition, JDBCPartitioningInfo, JDBCRelation} import org.apache.spark.sql.sources.{LogicalRelation, BaseRelation, DDLParser, DataSourceStrategy} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -171,14 +172,14 @@ class SQLContext(@transient val sparkContext: SparkContext) val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] val attributeSeq = schema.toAttributes val rowRDD = RDDConversions.productToRowRdd(rdd, schema) - new DataFrame(this, LogicalRDD(attributeSeq, rowRDD)(self)) + DataFrame(this, LogicalRDD(attributeSeq, rowRDD)(self)) } /** * Convert a [[BaseRelation]] created for external data sources into a [[DataFrame]]. */ def baseRelationToDataFrame(baseRelation: BaseRelation): DataFrame = { - new DataFrame(this, LogicalRelation(baseRelation)) + DataFrame(this, LogicalRelation(baseRelation)) } /** @@ -216,7 +217,7 @@ class SQLContext(@transient val sparkContext: SparkContext) // TODO: use MutableProjection when rowRDD is another DataFrame and the applied // schema differs from the existing schema on any field data type. val logicalPlan = LogicalRDD(schema.toAttributes, rowRDD)(self) - new DataFrame(this, logicalPlan) + DataFrame(this, logicalPlan) } /** @@ -243,7 +244,7 @@ class SQLContext(@transient val sparkContext: SparkContext) ) : Row } } - new DataFrame(this, LogicalRDD(attributeSeq, rowRdd)(this)) + DataFrame(this, LogicalRDD(attributeSeq, rowRdd)(this)) } /** @@ -262,7 +263,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * @group userf */ def parquetFile(path: String): DataFrame = - new DataFrame(this, parquet.ParquetRelation(path, Some(sparkContext.hadoopConfiguration), this)) + DataFrame(this, parquet.ParquetRelation(path, Some(sparkContext.hadoopConfiguration), this)) /** * Loads a JSON file (one object per line), returning the result as a [[DataFrame]]. @@ -334,6 +335,52 @@ class SQLContext(@transient val sparkContext: SparkContext) applySchema(rowRDD, appliedSchema) } + /** + * :: Experimental :: + * Construct an RDD representing the database table accessible via JDBC URL + * url named table. + */ + @Experimental + def jdbcRDD(url: String, table: String): DataFrame = { + jdbcRDD(url, table, null.asInstanceOf[JDBCPartitioningInfo]) + } + + /** + * :: Experimental :: + * Construct an RDD representing the database table accessible via JDBC URL + * url named table. The PartitioningInfo parameter + * gives the name of a column of integral type, a number of partitions, and + * advisory minimum and maximum values for the column. The RDD is + * partitioned according to said column. + */ + @Experimental + def jdbcRDD(url: String, table: String, partitioning: JDBCPartitioningInfo): + DataFrame = { + val parts = JDBCRelation.columnPartition(partitioning) + jdbcRDD(url, table, parts) + } + + /** + * :: Experimental :: + * Construct an RDD representing the database table accessible via JDBC URL + * url named table. The theParts parameter gives a list expressions + * suitable for inclusion in WHERE clauses; each one defines one partition + * of the RDD. + */ + @Experimental + def jdbcRDD(url: String, table: String, theParts: Array[String]): + DataFrame = { + val parts: Array[Partition] = theParts.zipWithIndex.map( + x => JDBCPartition(x._1, x._2).asInstanceOf[Partition]) + jdbcRDD(url, table, parts) + } + + private def jdbcRDD(url: String, table: String, parts: Array[Partition]): + DataFrame = { + val relation = JDBCRelation(url, table, parts)(this) + baseRelationToDataFrame(relation) + } + /** * Registers the given RDD as a temporary table in the catalog. Temporary tables exist only * during the lifetime of this instance of SQLContext. @@ -365,7 +412,7 @@ class SQLContext(@transient val sparkContext: SparkContext) */ def sql(sqlText: String): DataFrame = { if (conf.dialect == "sql") { - new DataFrame(this, parseSql(sqlText)) + DataFrame(this, parseSql(sqlText)) } else { sys.error(s"Unsupported SQL dialect: ${conf.dialect}") } @@ -373,7 +420,7 @@ class SQLContext(@transient val sparkContext: SparkContext) /** Returns the specified table as a [[DataFrame]]. */ def table(tableName: String): DataFrame = - new DataFrame(this, catalog.lookupRelation(Seq(tableName))) + DataFrame(this, catalog.lookupRelation(Seq(tableName))) protected[sql] class SparkPlanner extends SparkStrategies { val sparkContext: SparkContext = self.sparkContext @@ -462,7 +509,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * access to the intermediate phases of query execution for developers. */ @DeveloperApi - protected class QueryExecution(val logical: LogicalPlan) { + protected[sql] class QueryExecution(val logical: LogicalPlan) { lazy val analyzed: LogicalPlan = ExtractPythonUdfs(analyzer(logical)) lazy val withCachedData: LogicalPlan = cacheManager.useCachedData(analyzed) @@ -556,7 +603,7 @@ class SQLContext(@transient val sparkContext: SparkContext) iter.map { m => new GenericRow(m): Row} } - new DataFrame(this, LogicalRDD(schema.toAttributes, rowRdd)(self)) + DataFrame(this, LogicalRDD(schema.toAttributes, rowRdd)(self)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 6fba76c52171b..e1c9a2be7d20d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -138,7 +138,7 @@ case class CacheTableCommand( override def run(sqlContext: SQLContext) = { plan.foreach { logicalPlan => - sqlContext.registerRDDAsTable(new DataFrame(sqlContext, logicalPlan), tableName) + sqlContext.registerRDDAsTable(DataFrame(sqlContext, logicalPlan), tableName) } sqlContext.cacheTable(tableName) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DriverQuirks.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DriverQuirks.scala new file mode 100644 index 0000000000000..1704be7fcbd30 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DriverQuirks.scala @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import org.apache.spark.sql.types._ + +import java.sql.Types + + +/** + * Encapsulates workarounds for the extensions, quirks, and bugs in various + * databases. Lots of databases define types that aren't explicitly supported + * by the JDBC spec. Some JDBC drivers also report inaccurate + * information---for instance, BIT(n>1) being reported as a BIT type is quite + * common, even though BIT in JDBC is meant for single-bit values. Also, there + * does not appear to be a standard name for an unbounded string or binary + * type; we use BLOB and CLOB by default but override with database-specific + * alternatives when these are absent or do not behave correctly. + * + * Currently, the only thing DriverQuirks does is handle type mapping. + * `getCatalystType` is used when reading from a JDBC table and `getJDBCType` + * is used when writing to a JDBC table. If `getCatalystType` returns `null`, + * the default type handling is used for the given JDBC type. Similarly, + * if `getJDBCType` returns `(null, None)`, the default type handling is used + * for the given Catalyst type. + */ +private[sql] abstract class DriverQuirks { + def getCatalystType(sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): DataType + def getJDBCType(dt: DataType): (String, Option[Int]) +} + +private[sql] object DriverQuirks { + /** + * Fetch the DriverQuirks class corresponding to a given database url. + */ + def get(url: String): DriverQuirks = { + if (url.substring(0, 10).equals("jdbc:mysql")) { + new MySQLQuirks() + } else if (url.substring(0, 15).equals("jdbc:postgresql")) { + new PostgresQuirks() + } else { + new NoQuirks() + } + } +} + +private[sql] class NoQuirks extends DriverQuirks { + def getCatalystType(sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): DataType = + null + def getJDBCType(dt: DataType): (String, Option[Int]) = (null, None) +} + +private[sql] class PostgresQuirks extends DriverQuirks { + def getCatalystType(sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): DataType = { + if (sqlType == Types.BIT && typeName.equals("bit") && size != 1) { + BinaryType + } else if (sqlType == Types.OTHER && typeName.equals("cidr")) { + StringType + } else if (sqlType == Types.OTHER && typeName.equals("inet")) { + StringType + } else null + } + + def getJDBCType(dt: DataType): (String, Option[Int]) = dt match { + case StringType => ("TEXT", Some(java.sql.Types.CHAR)) + case BinaryType => ("BYTEA", Some(java.sql.Types.BINARY)) + case BooleanType => ("BOOLEAN", Some(java.sql.Types.BOOLEAN)) + case _ => (null, None) + } +} + +private[sql] class MySQLQuirks extends DriverQuirks { + def getCatalystType(sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): DataType = { + if (sqlType == Types.VARBINARY && typeName.equals("BIT") && size != 1) { + // This could instead be a BinaryType if we'd rather return bit-vectors of up to 64 bits as + // byte arrays instead of longs. + md.putLong("binarylong", 1) + LongType + } else if (sqlType == Types.BIT && typeName.equals("TINYINT")) { + BooleanType + } else null + } + def getJDBCType(dt: DataType): (String, Option[Int]) = (null, None) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala new file mode 100644 index 0000000000000..a2f94675fb5a3 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala @@ -0,0 +1,417 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import java.sql.{Connection, DatabaseMetaData, DriverManager, ResultSet, ResultSetMetaData, SQLException} +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.{Logging, Partition, SparkContext, TaskContext} +import org.apache.spark.rdd.RDD +import org.apache.spark.util.NextIterator +import org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion +import org.apache.spark.sql.catalyst.expressions.{Row, SpecificMutableRow} +import org.apache.spark.sql.types._ +import org.apache.spark.sql.sources._ + +private[sql] object JDBCRDD extends Logging { + /** + * Maps a JDBC type to a Catalyst type. This function is called only when + * the DriverQuirks class corresponding to your database driver returns null. + * + * @param sqlType - A field of java.sql.Types + * @return The Catalyst type corresponding to sqlType. + */ + private def getCatalystType(sqlType: Int): DataType = { + val answer = sqlType match { + case java.sql.Types.ARRAY => null + case java.sql.Types.BIGINT => LongType + case java.sql.Types.BINARY => BinaryType + case java.sql.Types.BIT => BooleanType // Per JDBC; Quirks handles quirky drivers. + case java.sql.Types.BLOB => BinaryType + case java.sql.Types.BOOLEAN => BooleanType + case java.sql.Types.CHAR => StringType + case java.sql.Types.CLOB => StringType + case java.sql.Types.DATALINK => null + case java.sql.Types.DATE => DateType + case java.sql.Types.DECIMAL => DecimalType.Unlimited + case java.sql.Types.DISTINCT => null + case java.sql.Types.DOUBLE => DoubleType + case java.sql.Types.FLOAT => FloatType + case java.sql.Types.INTEGER => IntegerType + case java.sql.Types.JAVA_OBJECT => null + case java.sql.Types.LONGNVARCHAR => StringType + case java.sql.Types.LONGVARBINARY => BinaryType + case java.sql.Types.LONGVARCHAR => StringType + case java.sql.Types.NCHAR => StringType + case java.sql.Types.NCLOB => StringType + case java.sql.Types.NULL => null + case java.sql.Types.NUMERIC => DecimalType.Unlimited + case java.sql.Types.OTHER => null + case java.sql.Types.REAL => DoubleType + case java.sql.Types.REF => StringType + case java.sql.Types.ROWID => LongType + case java.sql.Types.SMALLINT => IntegerType + case java.sql.Types.SQLXML => StringType + case java.sql.Types.STRUCT => StringType + case java.sql.Types.TIME => TimestampType + case java.sql.Types.TIMESTAMP => TimestampType + case java.sql.Types.TINYINT => IntegerType + case java.sql.Types.VARBINARY => BinaryType + case java.sql.Types.VARCHAR => StringType + case _ => null + } + + if (answer == null) throw new SQLException("Unsupported type " + sqlType) + answer + } + + /** + * Takes a (schema, table) specification and returns the table's Catalyst + * schema. + * + * @param url - The JDBC url to fetch information from. + * @param table - The table name of the desired table. This may also be a + * SQL query wrapped in parentheses. + * + * @return A StructType giving the table's Catalyst schema. + * @throws SQLException if the table specification is garbage. + * @throws SQLException if the table contains an unsupported type. + */ + def resolveTable(url: String, table: String): StructType = { + val quirks = DriverQuirks.get(url) + val conn: Connection = DriverManager.getConnection(url) + try { + val rs = conn.prepareStatement(s"SELECT * FROM $table WHERE 1=0").executeQuery() + try { + val rsmd = rs.getMetaData + val ncols = rsmd.getColumnCount + var fields = new Array[StructField](ncols); + var i = 0 + while (i < ncols) { + val columnName = rsmd.getColumnName(i + 1) + val dataType = rsmd.getColumnType(i + 1) + val typeName = rsmd.getColumnTypeName(i + 1) + val fieldSize = rsmd.getPrecision(i + 1) + val nullable = rsmd.isNullable(i + 1) != ResultSetMetaData.columnNoNulls + val metadata = new MetadataBuilder().putString("name", columnName) + var columnType = quirks.getCatalystType(dataType, typeName, fieldSize, metadata) + if (columnType == null) columnType = getCatalystType(dataType) + fields(i) = StructField(columnName, columnType, nullable, metadata.build()) + i = i + 1 + } + return new StructType(fields) + } finally { + rs.close() + } + } finally { + conn.close() + } + + throw new RuntimeException("This line is unreachable.") + } + + /** + * Prune all but the specified columns from the specified Catalyst schema. + * + * @param schema - The Catalyst schema of the master table + * @param columns - The list of desired columns + * + * @return A Catalyst schema corresponding to columns in the given order. + */ + private def pruneSchema(schema: StructType, columns: Array[String]): StructType = { + val fieldMap = Map(schema.fields map { x => x.metadata.getString("name") -> x }: _*) + new StructType(columns map { name => fieldMap(name) }) + } + + /** + * Given a driver string and an url, return a function that loads the + * specified driver string then returns a connection to the JDBC url. + * getConnector is run on the driver code, while the function it returns + * is run on the executor. + * + * @param driver - The class name of the JDBC driver for the given url. + * @param url - The JDBC url to connect to. + * + * @return A function that loads the driver and connects to the url. + */ + def getConnector(driver: String, url: String): () => Connection = { + () => { + try { + if (driver != null) Class.forName(driver) + } catch { + case e: ClassNotFoundException => { + logWarning(s"Couldn't find class $driver", e); + } + } + DriverManager.getConnection(url) + } + } + /** + * Build and return JDBCRDD from the given information. + * + * @param sc - Your SparkContext. + * @param schema - The Catalyst schema of the underlying database table. + * @param driver - The class name of the JDBC driver for the given url. + * @param url - The JDBC url to connect to. + * @param fqTable - The fully-qualified table name (or paren'd SQL query) to use. + * @param requiredColumns - The names of the columns to SELECT. + * @param filters - The filters to include in all WHERE clauses. + * @param parts - An array of JDBCPartitions specifying partition ids and + * per-partition WHERE clauses. + * + * @return An RDD representing "SELECT requiredColumns FROM fqTable". + */ + def scanTable(sc: SparkContext, + schema: StructType, + driver: String, + url: String, + fqTable: String, + requiredColumns: Array[String], + filters: Array[Filter], + parts: Array[Partition]): RDD[Row] = { + val prunedSchema = pruneSchema(schema, requiredColumns) + + return new JDBCRDD(sc, + getConnector(driver, url), + prunedSchema, + fqTable, + requiredColumns, + filters, + parts) + } +} + +/** + * An RDD representing a table in a database accessed via JDBC. Both the + * driver code and the workers must be able to access the database; the driver + * needs to fetch the schema while the workers need to fetch the data. + */ +private[sql] class JDBCRDD( + sc: SparkContext, + getConnection: () => Connection, + schema: StructType, + fqTable: String, + columns: Array[String], + filters: Array[Filter], + partitions: Array[Partition]) + extends RDD[Row](sc, Nil) { + + /** + * Retrieve the list of partitions corresponding to this RDD. + */ + override def getPartitions: Array[Partition] = partitions + + /** + * `columns`, but as a String suitable for injection into a SQL query. + */ + private val columnList: String = { + val sb = new StringBuilder() + columns.foreach(x => sb.append(",").append(x)) + if (sb.length == 0) "1" else sb.substring(1) + } + + /** + * Turns a single Filter into a String representing a SQL expression. + * Returns null for an unhandled filter. + */ + private def compileFilter(f: Filter): String = f match { + case EqualTo(attr, value) => s"$attr = $value" + case LessThan(attr, value) => s"$attr < $value" + case GreaterThan(attr, value) => s"$attr > $value" + case LessThanOrEqual(attr, value) => s"$attr <= $value" + case GreaterThanOrEqual(attr, value) => s"$attr >= $value" + case _ => null + } + + /** + * `filters`, but as a WHERE clause suitable for injection into a SQL query. + */ + private val filterWhereClause: String = { + val filterStrings = filters map compileFilter filter (_ != null) + if (filterStrings.size > 0) { + val sb = new StringBuilder("WHERE ") + filterStrings.foreach(x => sb.append(x).append(" AND ")) + sb.substring(0, sb.length - 5) + } else "" + } + + /** + * A WHERE clause representing both `filters`, if any, and the current partition. + */ + private def getWhereClause(part: JDBCPartition): String = { + if (part.whereClause != null && filterWhereClause.length > 0) { + filterWhereClause + " AND " + part.whereClause + } else if (part.whereClause != null) { + "WHERE " + part.whereClause + } else { + filterWhereClause + } + } + + // Each JDBC-to-Catalyst conversion corresponds to a tag defined here so that + // we don't have to potentially poke around in the Metadata once for every + // row. + // Is there a better way to do this? I'd rather be using a type that + // contains only the tags I define. + abstract class JDBCConversion + case object BooleanConversion extends JDBCConversion + case object DateConversion extends JDBCConversion + case object DecimalConversion extends JDBCConversion + case object DoubleConversion extends JDBCConversion + case object FloatConversion extends JDBCConversion + case object IntegerConversion extends JDBCConversion + case object LongConversion extends JDBCConversion + case object BinaryLongConversion extends JDBCConversion + case object StringConversion extends JDBCConversion + case object TimestampConversion extends JDBCConversion + case object BinaryConversion extends JDBCConversion + + /** + * Maps a StructType to a type tag list. + */ + def getConversions(schema: StructType): Array[JDBCConversion] = { + schema.fields.map(sf => sf.dataType match { + case BooleanType => BooleanConversion + case DateType => DateConversion + case DecimalType.Unlimited => DecimalConversion + case DoubleType => DoubleConversion + case FloatType => FloatConversion + case IntegerType => IntegerConversion + case LongType => + if (sf.metadata.contains("binarylong")) BinaryLongConversion else LongConversion + case StringType => StringConversion + case TimestampType => TimestampConversion + case BinaryType => BinaryConversion + case _ => throw new IllegalArgumentException(s"Unsupported field $sf") + }).toArray + } + + + /** + * Runs the SQL query against the JDBC driver. + */ + override def compute(thePart: Partition, context: TaskContext) = new Iterator[Row] { + var closed = false + var finished = false + var gotNext = false + var nextValue: Row = null + + context.addTaskCompletionListener{ context => close() } + val part = thePart.asInstanceOf[JDBCPartition] + val conn = getConnection() + + // H2's JDBC driver does not support the setSchema() method. We pass a + // fully-qualified table name in the SELECT statement. I don't know how to + // talk about a table in a completely portable way. + + val myWhereClause = getWhereClause(part) + + val sqlText = s"SELECT $columnList FROM $fqTable $myWhereClause" + val stmt = conn.prepareStatement(sqlText, + ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY) + val rs = stmt.executeQuery() + + val conversions = getConversions(schema) + val mutableRow = new SpecificMutableRow(schema.fields.map(x => x.dataType)) + + def getNext(): Row = { + if (rs.next()) { + var i = 0 + while (i < conversions.length) { + val pos = i + 1 + conversions(i) match { + case BooleanConversion => mutableRow.setBoolean(i, rs.getBoolean(pos)) + case DateConversion => mutableRow.update(i, rs.getDate(pos)) + case DecimalConversion => mutableRow.update(i, rs.getBigDecimal(pos)) + case DoubleConversion => mutableRow.setDouble(i, rs.getDouble(pos)) + case FloatConversion => mutableRow.setFloat(i, rs.getFloat(pos)) + case IntegerConversion => mutableRow.setInt(i, rs.getInt(pos)) + case LongConversion => mutableRow.setLong(i, rs.getLong(pos)) + case StringConversion => mutableRow.setString(i, rs.getString(pos)) + case TimestampConversion => mutableRow.update(i, rs.getTimestamp(pos)) + case BinaryConversion => mutableRow.update(i, rs.getBytes(pos)) + case BinaryLongConversion => { + val bytes = rs.getBytes(pos) + var ans = 0L + var j = 0 + while (j < bytes.size) { + ans = 256*ans + (255 & bytes(j)) + j = j + 1; + } + mutableRow.setLong(i, ans) + } + } + if (rs.wasNull) mutableRow.setNullAt(i) + i = i + 1 + } + mutableRow + } else { + finished = true + null.asInstanceOf[Row] + } + } + + def close() { + if (closed) return + try { + if (null != rs && ! rs.isClosed()) { + rs.close() + } + } catch { + case e: Exception => logWarning("Exception closing resultset", e) + } + try { + if (null != stmt && ! stmt.isClosed()) { + stmt.close() + } + } catch { + case e: Exception => logWarning("Exception closing statement", e) + } + try { + if (null != conn && ! conn.isClosed()) { + conn.close() + } + logInfo("closed connection") + } catch { + case e: Exception => logWarning("Exception closing connection", e) + } + } + + override def hasNext: Boolean = { + if (!finished) { + if (!gotNext) { + nextValue = getNext() + if (finished) { + close() + } + gotNext = true + } + } + !finished + } + + override def next(): Row = { + if (!hasNext) { + throw new NoSuchElementException("End of stream") + } + gotNext = false + nextValue + } + + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala new file mode 100644 index 0000000000000..e09125e406ba2 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import scala.collection.mutable.ArrayBuffer +import java.sql.DriverManager + +import org.apache.spark.Partition +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.sources._ + +/** + * Data corresponding to one partition of a JDBCRDD. + */ +private[sql] case class JDBCPartition(whereClause: String, idx: Int) extends Partition { + override def index: Int = idx +} + +/** + * Instructions on how to partition the table among workers. + */ +private[sql] case class JDBCPartitioningInfo( + column: String, + lowerBound: Long, + upperBound: Long, + numPartitions: Int) + +private[sql] object JDBCRelation { + /** + * Given a partitioning schematic (a column of integral type, a number of + * partitions, and upper and lower bounds on the column's value), generate + * WHERE clauses for each partition so that each row in the table appears + * exactly once. The parameters minValue and maxValue are advisory in that + * incorrect values may cause the partitioning to be poor, but no data + * will fail to be represented. + * + * @param column - Column name. Must refer to a column of integral type. + * @param numPartitions - Number of partitions + * @param minValue - Smallest value of column. Advisory. + * @param maxValue - Largest value of column. Advisory. + */ + def columnPartition(partitioning: JDBCPartitioningInfo): Array[Partition] = { + if (partitioning == null) return Array[Partition](JDBCPartition(null, 0)) + + val numPartitions = partitioning.numPartitions + val column = partitioning.column + if (numPartitions == 1) return Array[Partition](JDBCPartition(null, 0)) + // Overflow and silliness can happen if you subtract then divide. + // Here we get a little roundoff, but that's (hopefully) OK. + val stride: Long = (partitioning.upperBound / numPartitions + - partitioning.lowerBound / numPartitions) + var i: Int = 0 + var currentValue: Long = partitioning.lowerBound + var ans = new ArrayBuffer[Partition]() + while (i < numPartitions) { + val lowerBound = (if (i != 0) s"$column >= $currentValue" else null) + currentValue += stride + val upperBound = (if (i != numPartitions - 1) s"$column < $currentValue" else null) + val whereClause = (if (upperBound == null) lowerBound + else if (lowerBound == null) upperBound + else s"$lowerBound AND $upperBound") + ans += JDBCPartition(whereClause, i) + i = i + 1 + } + ans.toArray + } +} + +private[sql] class DefaultSource extends RelationProvider { + /** Returns a new base relation with the given parameters. */ + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String]): BaseRelation = { + val url = parameters.getOrElse("url", sys.error("Option 'url' not specified")) + val driver = parameters.getOrElse("driver", null) + val table = parameters.getOrElse("dbtable", sys.error("Option 'dbtable' not specified")) + val partitionColumn = parameters.getOrElse("partitionColumn", null) + val lowerBound = parameters.getOrElse("lowerBound", null) + val upperBound = parameters.getOrElse("upperBound", null) + val numPartitions = parameters.getOrElse("numPartitions", null) + + if (driver != null) Class.forName(driver) + + if ( partitionColumn != null + && (lowerBound == null || upperBound == null || numPartitions == null)) { + sys.error("Partitioning incompletely specified") + } + + val partitionInfo = if (partitionColumn == null) { + null + } else { + JDBCPartitioningInfo(partitionColumn, + lowerBound.toLong, upperBound.toLong, + numPartitions.toInt) + } + val parts = JDBCRelation.columnPartition(partitionInfo) + JDBCRelation(url, table, parts)(sqlContext) + } +} + +private[sql] case class JDBCRelation(url: String, + table: String, + parts: Array[Partition])( + @transient val sqlContext: SQLContext) + extends PrunedFilteredScan { + + override val schema = JDBCRDD.resolveTable(url, table) + + override def buildScan(requiredColumns: Array[String], filters: Array[Filter]) = { + val driver: String = DriverManager.getDriver(url).getClass.getCanonicalName + JDBCRDD.scanTable(sqlContext.sparkContext, + schema, + driver, url, + table, + requiredColumns, filters, + parts) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JavaJDBCTrampoline.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JavaJDBCTrampoline.scala new file mode 100644 index 0000000000000..86bb67ec74256 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JavaJDBCTrampoline.scala @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import org.apache.spark.sql.DataFrame + +private[jdbc] class JavaJDBCTrampoline { + def createJDBCTable(rdd: DataFrame, url: String, table: String, allowExisting: Boolean) { + rdd.createJDBCTable(url, table, allowExisting); + } + + def insertIntoJDBC(rdd: DataFrame, url: String, table: String, overwrite: Boolean) { + rdd.insertIntoJDBC(url, table, overwrite); + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala new file mode 100644 index 0000000000000..34a83f0a5dad8 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala @@ -0,0 +1,235 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import java.sql.{Connection, DriverManager, PreparedStatement} +import org.apache.spark.{Logging, Partition} +import org.apache.spark.sql._ +import org.apache.spark.sql.sources.LogicalRelation + +import org.apache.spark.sql.jdbc.{JDBCPartitioningInfo, JDBCRelation, JDBCPartition} +import org.apache.spark.sql.types._ + +package object jdbc { + object JDBCWriteDetails extends Logging { + /** + * Returns a PreparedStatement that inserts a row into table via conn. + */ + private def insertStatement(conn: Connection, table: String, rddSchema: StructType): + PreparedStatement = { + val sql = new StringBuilder(s"INSERT INTO $table VALUES (") + var fieldsLeft = rddSchema.fields.length + while (fieldsLeft > 0) { + sql.append("?") + if (fieldsLeft > 1) sql.append(", ") else sql.append(")") + fieldsLeft = fieldsLeft - 1 + } + conn.prepareStatement(sql.toString) + } + + /** + * Saves a partition of a DataFrame to the JDBC database. This is done in + * a single database transaction in order to avoid repeatedly inserting + * data as much as possible. + * + * It is still theoretically possible for rows in a DataFrame to be + * inserted into the database more than once if a stage somehow fails after + * the commit occurs but before the stage can return successfully. + * + * This is not a closure inside saveTable() because apparently cosmetic + * implementation changes elsewhere might easily render such a closure + * non-Serializable. Instead, we explicitly close over all variables that + * are used. + */ + private[jdbc] def savePartition(url: String, table: String, iterator: Iterator[Row], + rddSchema: StructType, nullTypes: Array[Int]): Iterator[Byte] = { + val conn = DriverManager.getConnection(url) + var committed = false + try { + conn.setAutoCommit(false) // Everything in the same db transaction. + val stmt = insertStatement(conn, table, rddSchema) + try { + while (iterator.hasNext) { + val row = iterator.next() + val numFields = rddSchema.fields.length + var i = 0 + while (i < numFields) { + if (row.isNullAt(i)) { + stmt.setNull(i + 1, nullTypes(i)) + } else { + rddSchema.fields(i).dataType match { + case IntegerType => stmt.setInt(i + 1, row.getInt(i)) + case LongType => stmt.setLong(i + 1, row.getLong(i)) + case DoubleType => stmt.setDouble(i + 1, row.getDouble(i)) + case FloatType => stmt.setFloat(i + 1, row.getFloat(i)) + case ShortType => stmt.setInt(i + 1, row.getShort(i)) + case ByteType => stmt.setInt(i + 1, row.getByte(i)) + case BooleanType => stmt.setBoolean(i + 1, row.getBoolean(i)) + case StringType => stmt.setString(i + 1, row.getString(i)) + case BinaryType => stmt.setBytes(i + 1, row.getAs[Array[Byte]](i)) + case TimestampType => stmt.setTimestamp(i + 1, row.getAs[java.sql.Timestamp](i)) + case DateType => stmt.setDate(i + 1, row.getAs[java.sql.Date](i)) + case DecimalType.Unlimited => stmt.setBigDecimal(i + 1, + row.getAs[java.math.BigDecimal](i)) + case _ => throw new IllegalArgumentException( + s"Can't translate non-null value for field $i") + } + } + i = i + 1 + } + stmt.executeUpdate() + } + } finally { + stmt.close() + } + conn.commit() + committed = true + } finally { + if (!committed) { + // The stage must fail. We got here through an exception path, so + // let the exception through unless rollback() or close() want to + // tell the user about another problem. + conn.rollback() + conn.close() + } else { + // The stage must succeed. We cannot propagate any exception close() might throw. + try { + conn.close() + } catch { + case e: Exception => logWarning("Transaction succeeded, but closing failed", e) + } + } + } + Array[Byte]().iterator + } + } + + /** + * Make it so that you can call createJDBCTable and insertIntoJDBC on a DataFrame. + */ + implicit class JDBCDataFrame(rdd: DataFrame) { + /** + * Compute the schema string for this RDD. + */ + private def schemaString(url: String): String = { + val sb = new StringBuilder() + val quirks = DriverQuirks.get(url) + rdd.schema.fields foreach { field => { + val name = field.name + var typ: String = quirks.getJDBCType(field.dataType)._1 + if (typ == null) typ = field.dataType match { + case IntegerType => "INTEGER" + case LongType => "BIGINT" + case DoubleType => "DOUBLE PRECISION" + case FloatType => "REAL" + case ShortType => "INTEGER" + case ByteType => "BYTE" + case BooleanType => "BIT(1)" + case StringType => "TEXT" + case BinaryType => "BLOB" + case TimestampType => "TIMESTAMP" + case DateType => "DATE" + case DecimalType.Unlimited => "DECIMAL(40,20)" + case _ => throw new IllegalArgumentException(s"Don't know how to save $field to JDBC") + } + val nullable = if (field.nullable) "" else "NOT NULL" + sb.append(s", $name $typ $nullable") + }} + if (sb.length < 2) "" else sb.substring(2) + } + + /** + * Saves the RDD to the database in a single transaction. + */ + private def saveTable(url: String, table: String) { + val quirks = DriverQuirks.get(url) + var nullTypes: Array[Int] = rdd.schema.fields.map(field => { + var nullType: Option[Int] = quirks.getJDBCType(field.dataType)._2 + if (nullType.isEmpty) { + field.dataType match { + case IntegerType => java.sql.Types.INTEGER + case LongType => java.sql.Types.BIGINT + case DoubleType => java.sql.Types.DOUBLE + case FloatType => java.sql.Types.REAL + case ShortType => java.sql.Types.INTEGER + case ByteType => java.sql.Types.INTEGER + case BooleanType => java.sql.Types.BIT + case StringType => java.sql.Types.CLOB + case BinaryType => java.sql.Types.BLOB + case TimestampType => java.sql.Types.TIMESTAMP + case DateType => java.sql.Types.DATE + case DecimalType.Unlimited => java.sql.Types.DECIMAL + case _ => throw new IllegalArgumentException( + s"Can't translate null value for field $field") + } + } else nullType.get + }).toArray + + val rddSchema = rdd.schema + rdd.mapPartitions(iterator => JDBCWriteDetails.savePartition( + url, table, iterator, rddSchema, nullTypes)).collect() + } + + /** + * Save this RDD to a JDBC database at `url` under the table name `table`. + * This will run a `CREATE TABLE` and a bunch of `INSERT INTO` statements. + * If you pass `true` for `allowExisting`, it will drop any table with the + * given name; if you pass `false`, it will throw if the table already + * exists. + */ + def createJDBCTable(url: String, table: String, allowExisting: Boolean) { + val conn = DriverManager.getConnection(url) + try { + if (allowExisting) { + val sql = s"DROP TABLE IF EXISTS $table" + conn.prepareStatement(sql).executeUpdate() + } + val schema = schemaString(url) + val sql = s"CREATE TABLE $table ($schema)" + conn.prepareStatement(sql).executeUpdate() + } finally { + conn.close() + } + saveTable(url, table) + } + + /** + * Save this RDD to a JDBC database at `url` under the table name `table`. + * Assumes the table already exists and has a compatible schema. If you + * pass `true` for `overwrite`, it will `TRUNCATE` the table before + * performing the `INSERT`s. + * + * The table must already exist on the database. It must have a schema + * that is compatible with the schema of this RDD; inserting the rows of + * the RDD in order via the simple statement + * `INSERT INTO table VALUES (?, ?, ..., ?)` should not fail. + */ + def insertIntoJDBC(url: String, table: String, overwrite: Boolean) { + if (overwrite) { + val conn = DriverManager.getConnection(url) + try { + val sql = s"TRUNCATE TABLE $table" + conn.prepareStatement(sql).executeUpdate() + } finally { + conn.close() + } + } + saveTable(url, table) + } + } // implicit class JDBCDataFrame +} // package object jdbc diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index b7c721f8c0691..b1bbe0f89af73 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -231,7 +231,7 @@ private [sql] case class CreateTempTableUsing( def run(sqlContext: SQLContext) = { val resolved = ResolvedDataSource(sqlContext, userSpecifiedSchema, provider, options) sqlContext.registerRDDAsTable( - new DataFrame(sqlContext, LogicalRelation(resolved.relation)), tableName) + DataFrame(sqlContext, LogicalRelation(resolved.relation)), tableName) Seq.empty } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala index 906455dd40c0d..4e1ec38bd0158 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala @@ -41,7 +41,7 @@ object TestSQLContext * construct [[DataFrame]] directly out of local data without relying on implicits. */ protected[sql] implicit def logicalPlanToSparkQuery(plan: LogicalPlan): DataFrame = { - new DataFrame(this, plan) + DataFrame(this, plan) } } diff --git a/sql/core/src/test/java/org/apache/spark/sql/jdbc/JavaJDBCTest.java b/sql/core/src/test/java/org/apache/spark/sql/jdbc/JavaJDBCTest.java new file mode 100644 index 0000000000000..80bd74f5b5525 --- /dev/null +++ b/sql/core/src/test/java/org/apache/spark/sql/jdbc/JavaJDBCTest.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc; + +import org.junit.*; +import static org.junit.Assert.*; +import java.sql.Connection; +import java.sql.DriverManager; + +import org.apache.spark.SparkEnv; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.api.java.*; +import org.apache.spark.sql.test.TestSQLContext$; + +public class JavaJDBCTest { + static String url = "jdbc:h2:mem:testdb1"; + + static Connection conn = null; + + // This variable will always be null if TestSQLContext is intact when running + // these tests. Some Java tests do not play nicely with others, however; + // they create a SparkContext of their own at startup and stop it at exit. + // This renders TestSQLContext inoperable, meaning we have to do the same + // thing. If this variable is nonnull, that means we allocated a + // SparkContext of our own and that we need to stop it at teardown. + static JavaSparkContext localSparkContext = null; + + static SQLContext sql = TestSQLContext$.MODULE$; + + @Before + public void beforeTest() throws Exception { + if (SparkEnv.get() == null) { // A previous test destroyed TestSQLContext. + localSparkContext = new JavaSparkContext("local", "JavaAPISuite"); + sql = new SQLContext(localSparkContext); + } + Class.forName("org.h2.Driver"); + conn = DriverManager.getConnection(url); + conn.prepareStatement("create schema test").executeUpdate(); + conn.prepareStatement("create table test.people (name TEXT(32) NOT NULL, theid INTEGER NOT NULL)").executeUpdate(); + conn.prepareStatement("insert into test.people values ('fred', 1)").executeUpdate(); + conn.prepareStatement("insert into test.people values ('mary', 2)").executeUpdate(); + conn.prepareStatement("insert into test.people values ('joe', 3)").executeUpdate(); + conn.commit(); + } + + @After + public void afterTest() throws Exception { + if (localSparkContext != null) { + localSparkContext.stop(); + localSparkContext = null; + } + try { + conn.close(); + } finally { + conn = null; + } + } + + @Test + public void basicTest() throws Exception { + DataFrame rdd = JDBCUtils.jdbcRDD(sql, url, "TEST.PEOPLE"); + Row[] rows = rdd.collect(); + assertEquals(rows.length, 3); + } + + @Test + public void partitioningTest() throws Exception { + String[] parts = new String[2]; + parts[0] = "THEID < 2"; + parts[1] = "THEID = 2"; // Deliberately forget about one of them. + DataFrame rdd = JDBCUtils.jdbcRDD(sql, url, "TEST.PEOPLE", parts); + Row[] rows = rdd.collect(); + assertEquals(rows.length, 2); + } + + @Test + public void writeTest() throws Exception { + DataFrame rdd = JDBCUtils.jdbcRDD(sql, url, "TEST.PEOPLE"); + JDBCUtils.createJDBCTable(rdd, url, "TEST.PEOPLECOPY", false); + DataFrame rdd2 = JDBCUtils.jdbcRDD(sql, url, "TEST.PEOPLECOPY"); + Row[] rows = rdd2.collect(); + assertEquals(rows.length, 3); + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index 2d464c2b53d79..fa4cdecbcb340 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -27,6 +27,45 @@ class ColumnExpressionSuite extends QueryTest { // TODO: Add test cases for bitwise operations. + test("computability check") { + def shouldBeComputable(c: Column): Unit = assert(c.isComputable === true) + + def shouldNotBeComputable(c: Column): Unit = { + assert(c.isComputable === false) + intercept[UnsupportedOperationException] { c.head() } + } + + shouldBeComputable(testData2("a")) + shouldBeComputable(testData2("b")) + + shouldBeComputable(testData2("a") + testData2("b")) + shouldBeComputable(testData2("a") + testData2("b") + 1) + + shouldBeComputable(-testData2("a")) + shouldBeComputable(!testData2("a")) + + shouldBeComputable(testData2.select(($"a" + 1).as("c"))("c") + testData2("b")) + shouldBeComputable( + testData2.select(($"a" + 1).as("c"))("c") + testData2.select(($"b" / 2).as("d"))("d")) + shouldBeComputable( + testData2.select(($"a" + 1).as("c")).select(($"c" + 2).as("d"))("d") + testData2("b")) + + // Literals and unresolved columns should not be computable. + shouldNotBeComputable(col("1")) + shouldNotBeComputable(col("1") + 2) + shouldNotBeComputable(lit(100)) + shouldNotBeComputable(lit(100) + 10) + shouldNotBeComputable(-col("1")) + shouldNotBeComputable(!col("1")) + + // Getting data from different frames should not be computable. + shouldNotBeComputable(testData2("a") + testData("key")) + shouldNotBeComputable(testData2("a") + 1 + testData("key")) + + // Aggregate functions alone should not be computable. + shouldNotBeComputable(sum(testData2("a"))) + } + test("star") { checkAnswer(testData.select($"*"), testData.collect().toSeq) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index df343adc793bd..f6b65a81ce05e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -28,6 +28,19 @@ import scala.language.postfixOps class DataFrameSuite extends QueryTest { import org.apache.spark.sql.TestData._ + test("analysis error should be eagerly reported") { + intercept[Exception] { testData.select('nonExistentName) } + intercept[Exception] { + testData.groupBy('key).agg(Map("nonExistentName" -> "sum")) + } + intercept[Exception] { + testData.groupBy("nonExistentName").agg(Map("key" -> "sum")) + } + intercept[Exception] { + testData.groupBy($"abcd").agg(Map("key" -> "sum")) + } + } + test("table scan") { checkAnswer( testData, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index d82c34316cefa..e18ba287e8683 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -807,13 +807,11 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { test("throw errors for non-aggregate attributes with aggregation") { def checkAggregation(query: String, isInvalidQuery: Boolean = true) { - val logicalPlan = sql(query).queryExecution.logical - if (isInvalidQuery) { val e = intercept[TreeNodeException[LogicalPlan]](sql(query).queryExecution.analyzed) assert( e.getMessage.startsWith("Expression not in GROUP BY"), - "Non-aggregate attribute(s) not detected\n" + logicalPlan) + "Non-aggregate attribute(s) not detected\n") } else { // Should not throw sql(query).queryExecution.analyzed @@ -821,7 +819,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } checkAggregation("SELECT key, COUNT(*) FROM testData") - checkAggregation("SELECT COUNT(key), COUNT(*) FROM testData", false) + checkAggregation("SELECT COUNT(key), COUNT(*) FROM testData", isInvalidQuery = false) checkAggregation("SELECT value, COUNT(*) FROM testData GROUP BY key") checkAggregation("SELECT COUNT(value), SUM(key) FROM testData GROUP BY key", false) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/DockerHacks.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/DockerHacks.scala new file mode 100644 index 0000000000000..f332cb389f339 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/DockerHacks.scala @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import scala.collection.mutable.MutableList + +import com.spotify.docker.client._ + +/** + * A factory and morgue for DockerClient objects. In the DockerClient we use, + * calling close() closes the desired DockerClient but also renders all other + * DockerClients inoperable. This is inconvenient if we have more than one + * open, such as during tests. + */ +object DockerClientFactory { + var numClients: Int = 0 + val zombies = new MutableList[DockerClient]() + + def get(): DockerClient = { + this.synchronized { + numClients = numClients + 1 + DefaultDockerClient.fromEnv.build() + } + } + + def close(dc: DockerClient) { + this.synchronized { + numClients = numClients - 1 + zombies += dc + if (numClients == 0) { + zombies.foreach(_.close()) + zombies.clear() + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala new file mode 100644 index 0000000000000..d25c1390db15c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -0,0 +1,248 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import java.math.BigDecimal +import org.apache.spark.sql.test._ +import org.scalatest.{FunSuite, BeforeAndAfter} +import java.sql.DriverManager +import TestSQLContext._ + +class JDBCSuite extends FunSuite with BeforeAndAfter { + val url = "jdbc:h2:mem:testdb0" + var conn: java.sql.Connection = null + + val testBytes = Array[Byte](99.toByte, 134.toByte, 135.toByte, 200.toByte, 205.toByte) + + before { + Class.forName("org.h2.Driver") + conn = DriverManager.getConnection(url) + conn.prepareStatement("create schema test").executeUpdate() + conn.prepareStatement("create table test.people (name TEXT(32) NOT NULL, theid INTEGER NOT NULL)").executeUpdate() + conn.prepareStatement("insert into test.people values ('fred', 1)").executeUpdate() + conn.prepareStatement("insert into test.people values ('mary', 2)").executeUpdate() + conn.prepareStatement("insert into test.people values ('joe', 3)").executeUpdate() + conn.commit() + + sql( + s""" + |CREATE TEMPORARY TABLE foobar + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$url', dbtable 'TEST.PEOPLE') + """.stripMargin.replaceAll("\n", " ")) + + sql( + s""" + |CREATE TEMPORARY TABLE parts + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$url', dbtable 'TEST.PEOPLE', + |partitionColumn 'THEID', lowerBound '1', upperBound '4', numPartitions '3') + """.stripMargin.replaceAll("\n", " ")) + + conn.prepareStatement("create table test.inttypes (a INT, b BOOLEAN, c TINYINT, " + + "d SMALLINT, e BIGINT)").executeUpdate() + conn.prepareStatement("insert into test.inttypes values (1, false, 3, 4, 1234567890123)" + ).executeUpdate() + conn.prepareStatement("insert into test.inttypes values (null, null, null, null, null)" + ).executeUpdate() + conn.commit() + sql( + s""" + |CREATE TEMPORARY TABLE inttypes + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$url', dbtable 'TEST.INTTYPES') + """.stripMargin.replaceAll("\n", " ")) + + conn.prepareStatement("create table test.strtypes (a BINARY(20), b VARCHAR(20), " + + "c VARCHAR_IGNORECASE(20), d CHAR(20), e BLOB, f CLOB)").executeUpdate() + var stmt = conn.prepareStatement("insert into test.strtypes values (?, ?, ?, ?, ?, ?)") + stmt.setBytes(1, testBytes) + stmt.setString(2, "Sensitive") + stmt.setString(3, "Insensitive") + stmt.setString(4, "Twenty-byte CHAR") + stmt.setBytes(5, testBytes) + stmt.setString(6, "I am a clob!") + stmt.executeUpdate() + sql( + s""" + |CREATE TEMPORARY TABLE strtypes + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$url', dbtable 'TEST.STRTYPES') + """.stripMargin.replaceAll("\n", " ")) + + conn.prepareStatement("create table test.timetypes (a TIME, b DATE, c TIMESTAMP)" + ).executeUpdate() + conn.prepareStatement("insert into test.timetypes values ('12:34:56', " + + "'1996-01-01', '2002-02-20 11:22:33.543543543')").executeUpdate() + conn.commit() + sql( + s""" + |CREATE TEMPORARY TABLE timetypes + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$url', dbtable 'TEST.TIMETYPES') + """.stripMargin.replaceAll("\n", " ")) + + + conn.prepareStatement("create table test.flttypes (a DOUBLE, b REAL, c DECIMAL(40, 20))" + ).executeUpdate() + conn.prepareStatement("insert into test.flttypes values (" + + "1.0000000000000002220446049250313080847263336181640625, " + + "1.00000011920928955078125, " + + "123456789012345.543215432154321)").executeUpdate() + conn.commit() + sql( + s""" + |CREATE TEMPORARY TABLE flttypes + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$url', dbtable 'TEST.FLTTYPES') + """.stripMargin.replaceAll("\n", " ")) + + // Untested: IDENTITY, OTHER, UUID, ARRAY, and GEOMETRY types. + } + + after { + conn.close() + } + + test("SELECT *") { + assert(sql("SELECT * FROM foobar").collect().size == 3) + } + + test("SELECT * WHERE (simple predicates)") { + assert(sql("SELECT * FROM foobar WHERE THEID < 1").collect().size == 0) + assert(sql("SELECT * FROM foobar WHERE THEID != 2").collect().size == 2) + assert(sql("SELECT * FROM foobar WHERE THEID = 1").collect().size == 1) + } + + test("SELECT first field") { + val names = sql("SELECT NAME FROM foobar").collect().map(x => x.getString(0)).sortWith(_ < _) + assert(names.size == 3) + assert(names(0).equals("fred")) + assert(names(1).equals("joe")) + assert(names(2).equals("mary")) + } + + test("SELECT second field") { + val ids = sql("SELECT THEID FROM foobar").collect().map(x => x.getInt(0)).sortWith(_ < _) + assert(ids.size == 3) + assert(ids(0) == 1) + assert(ids(1) == 2) + assert(ids(2) == 3) + } + + test("SELECT * partitioned") { + assert(sql("SELECT * FROM parts").collect().size == 3) + } + + test("SELECT WHERE (simple predicates) partitioned") { + assert(sql("SELECT * FROM parts WHERE THEID < 1").collect().size == 0) + assert(sql("SELECT * FROM parts WHERE THEID != 2").collect().size == 2) + assert(sql("SELECT THEID FROM parts WHERE THEID = 1").collect().size == 1) + } + + test("SELECT second field partitioned") { + val ids = sql("SELECT THEID FROM parts").collect().map(x => x.getInt(0)).sortWith(_ < _) + assert(ids.size == 3) + assert(ids(0) == 1) + assert(ids(1) == 2) + assert(ids(2) == 3) + } + + test("Basic API") { + assert(TestSQLContext.jdbcRDD(url, "TEST.PEOPLE").collect.size == 3) + } + + test("Partitioning via JDBCPartitioningInfo API") { + val parts = JDBCPartitioningInfo("THEID", 0, 4, 3) + assert(TestSQLContext.jdbcRDD(url, "TEST.PEOPLE", parts).collect.size == 3) + } + + test("Partitioning via list-of-where-clauses API") { + val parts = Array[String]("THEID < 2", "THEID >= 2") + assert(TestSQLContext.jdbcRDD(url, "TEST.PEOPLE", parts).collect.size == 3) + } + + test("H2 integral types") { + val rows = sql("SELECT * FROM inttypes WHERE A IS NOT NULL").collect() + assert(rows.size == 1) + assert(rows(0).getInt(0) == 1) + assert(rows(0).getBoolean(1) == false) + assert(rows(0).getInt(2) == 3) + assert(rows(0).getInt(3) == 4) + assert(rows(0).getLong(4) == 1234567890123L) + } + + test("H2 null entries") { + val rows = sql("SELECT * FROM inttypes WHERE A IS NULL").collect() + assert(rows.size == 1) + assert(rows(0).isNullAt(0)) + assert(rows(0).isNullAt(1)) + assert(rows(0).isNullAt(2)) + assert(rows(0).isNullAt(3)) + assert(rows(0).isNullAt(4)) + } + + test("H2 string types") { + val rows = sql("SELECT * FROM strtypes").collect() + assert(rows(0).getAs[Array[Byte]](0).sameElements(testBytes)) + assert(rows(0).getString(1).equals("Sensitive")) + assert(rows(0).getString(2).equals("Insensitive")) + assert(rows(0).getString(3).equals("Twenty-byte CHAR")) + assert(rows(0).getAs[Array[Byte]](4).sameElements(testBytes)) + assert(rows(0).getString(5).equals("I am a clob!")) + } + + test("H2 time types") { + val rows = sql("SELECT * FROM timetypes").collect() + assert(rows(0).getAs[java.sql.Timestamp](0).getHours == 12) + assert(rows(0).getAs[java.sql.Timestamp](0).getMinutes == 34) + assert(rows(0).getAs[java.sql.Timestamp](0).getSeconds == 56) + assert(rows(0).getAs[java.sql.Date](1).getYear == 96) + assert(rows(0).getAs[java.sql.Date](1).getMonth == 0) + assert(rows(0).getAs[java.sql.Date](1).getDate == 1) + assert(rows(0).getAs[java.sql.Timestamp](2).getYear == 102) + assert(rows(0).getAs[java.sql.Timestamp](2).getMonth == 1) + assert(rows(0).getAs[java.sql.Timestamp](2).getDate == 20) + assert(rows(0).getAs[java.sql.Timestamp](2).getHours == 11) + assert(rows(0).getAs[java.sql.Timestamp](2).getMinutes == 22) + assert(rows(0).getAs[java.sql.Timestamp](2).getSeconds == 33) + assert(rows(0).getAs[java.sql.Timestamp](2).getNanos == 543543543) + } + + test("H2 floating-point types") { + val rows = sql("SELECT * FROM flttypes").collect() + assert(rows(0).getDouble(0) == 1.00000000000000022) // Yes, I meant ==. + assert(rows(0).getDouble(1) == 1.00000011920928955) // Yes, I meant ==. + assert(rows(0).getAs[BigDecimal](2) + .equals(new BigDecimal("123456789012345.54321543215432100000"))) + } + + + test("SQL query as table name") { + sql( + s""" + |CREATE TEMPORARY TABLE hack + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$url', dbtable '(SELECT B, B*B FROM TEST.FLTTYPES)') + """.stripMargin.replaceAll("\n", " ")) + val rows = sql("SELECT * FROM hack").collect() + assert(rows(0).getDouble(0) == 1.00000011920928955) // Yes, I meant ==. + // For some reason, H2 computes this square incorrectly... + assert(math.abs(rows(0).getDouble(1) - 1.00000023841859331) < 1e-12) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala new file mode 100644 index 0000000000000..e581ac9b50c2b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import java.math.BigDecimal +import org.apache.spark.sql.Row +import org.apache.spark.sql.types._ +import org.apache.spark.sql.test._ +import org.scalatest.{FunSuite, BeforeAndAfter} +import java.sql.DriverManager +import TestSQLContext._ + +class JDBCWriteSuite extends FunSuite with BeforeAndAfter { + val url = "jdbc:h2:mem:testdb2" + var conn: java.sql.Connection = null + + before { + Class.forName("org.h2.Driver") + conn = DriverManager.getConnection(url) + conn.prepareStatement("create schema test").executeUpdate() + } + + after { + conn.close() + } + + val sc = TestSQLContext.sparkContext + + val arr2x2 = Array[Row](Row.apply("dave", 42), Row.apply("mary", 222)) + val arr1x2 = Array[Row](Row.apply("fred", 3)) + val schema2 = StructType( + StructField("name", StringType) :: + StructField("id", IntegerType) :: Nil) + + val arr2x3 = Array[Row](Row.apply("dave", 42, 1), Row.apply("mary", 222, 2)) + val schema3 = StructType( + StructField("name", StringType) :: + StructField("id", IntegerType) :: + StructField("seq", IntegerType) :: Nil) + + test("Basic CREATE") { + val srdd = TestSQLContext.applySchema(sc.parallelize(arr2x2), schema2) + + srdd.createJDBCTable(url, "TEST.BASICCREATETEST", false) + assert(2 == TestSQLContext.jdbcRDD(url, "TEST.BASICCREATETEST").count) + assert(2 == TestSQLContext.jdbcRDD(url, "TEST.BASICCREATETEST").collect()(0).length) + } + + test("CREATE with overwrite") { + val srdd = TestSQLContext.applySchema(sc.parallelize(arr2x3), schema3) + val srdd2 = TestSQLContext.applySchema(sc.parallelize(arr1x2), schema2) + + srdd.createJDBCTable(url, "TEST.DROPTEST", false) + assert(2 == TestSQLContext.jdbcRDD(url, "TEST.DROPTEST").count) + assert(3 == TestSQLContext.jdbcRDD(url, "TEST.DROPTEST").collect()(0).length) + + srdd2.createJDBCTable(url, "TEST.DROPTEST", true) + assert(1 == TestSQLContext.jdbcRDD(url, "TEST.DROPTEST").count) + assert(2 == TestSQLContext.jdbcRDD(url, "TEST.DROPTEST").collect()(0).length) + } + + test("CREATE then INSERT to append") { + val srdd = TestSQLContext.applySchema(sc.parallelize(arr2x2), schema2) + val srdd2 = TestSQLContext.applySchema(sc.parallelize(arr1x2), schema2) + + srdd.createJDBCTable(url, "TEST.APPENDTEST", false) + srdd2.insertIntoJDBC(url, "TEST.APPENDTEST", false) + assert(3 == TestSQLContext.jdbcRDD(url, "TEST.APPENDTEST").count) + assert(2 == TestSQLContext.jdbcRDD(url, "TEST.APPENDTEST").collect()(0).length) + } + + test("CREATE then INSERT to truncate") { + val srdd = TestSQLContext.applySchema(sc.parallelize(arr2x2), schema2) + val srdd2 = TestSQLContext.applySchema(sc.parallelize(arr1x2), schema2) + + srdd.createJDBCTable(url, "TEST.TRUNCATETEST", false) + srdd2.insertIntoJDBC(url, "TEST.TRUNCATETEST", true) + assert(1 == TestSQLContext.jdbcRDD(url, "TEST.TRUNCATETEST").count) + assert(2 == TestSQLContext.jdbcRDD(url, "TEST.TRUNCATETEST").collect()(0).length) + } + + test("Incompatible INSERT to append") { + val srdd = TestSQLContext.applySchema(sc.parallelize(arr2x2), schema2) + val srdd2 = TestSQLContext.applySchema(sc.parallelize(arr2x3), schema3) + + srdd.createJDBCTable(url, "TEST.INCOMPATIBLETEST", false) + intercept[org.apache.spark.SparkException] { + srdd2.insertIntoJDBC(url, "TEST.INCOMPATIBLETEST", true) + } + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala new file mode 100644 index 0000000000000..89920f2650c3a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala @@ -0,0 +1,235 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import java.math.BigDecimal +import java.sql.{Date, DriverManager, Timestamp} +import com.spotify.docker.client.{DefaultDockerClient, DockerClient} +import com.spotify.docker.client.messages.ContainerConfig +import org.scalatest.{FunSuite, BeforeAndAfterAll, Ignore} + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkContext._ +import org.apache.spark.sql._ +import org.apache.spark.sql.test._ +import TestSQLContext._ + +import org.apache.spark.sql.jdbc._ + +class MySQLDatabase { + val docker: DockerClient = DockerClientFactory.get() + val containerId = { + println("Pulling mysql") + docker.pull("mysql") + println("Configuring container") + val config = (ContainerConfig.builder().image("mysql") + .env("MYSQL_ROOT_PASSWORD=rootpass") + .build()) + println("Creating container") + val id = docker.createContainer(config).id + println("Starting container " + id) + docker.startContainer(id) + id + } + val ip = docker.inspectContainer(containerId).networkSettings.ipAddress + + def close() { + try { + println("Killing container " + containerId) + docker.killContainer(containerId) + println("Removing container " + containerId) + docker.removeContainer(containerId) + println("Closing docker client") + DockerClientFactory.close(docker) + } catch { + case e: Exception => { + println(e) + println("You may need to clean this up manually.") + throw e + } + } + } +} + +@Ignore class MySQLIntegration extends FunSuite with BeforeAndAfterAll { + var ip: String = null + + def url(ip: String): String = url(ip, "mysql") + def url(ip: String, db: String): String = s"jdbc:mysql://$ip:3306/$db?user=root&password=rootpass" + + def waitForDatabase(ip: String, maxMillis: Long) { + println("Waiting for database to start up.") + val before = System.currentTimeMillis() + var lastException: java.sql.SQLException = null + while (true) { + if (System.currentTimeMillis() > before + maxMillis) { + throw new java.sql.SQLException(s"Database not up after $maxMillis ms.", lastException) + } + try { + val conn = java.sql.DriverManager.getConnection(url(ip)) + conn.close() + println("Database is up.") + return; + } catch { + case e: java.sql.SQLException => { + lastException = e + java.lang.Thread.sleep(250) + } + } + } + } + + def setupDatabase(ip: String) { + val conn = java.sql.DriverManager.getConnection(url(ip)) + try { + conn.prepareStatement("CREATE DATABASE foo").executeUpdate() + conn.prepareStatement("CREATE TABLE foo.tbl (x INTEGER, y TEXT(8))").executeUpdate() + conn.prepareStatement("INSERT INTO foo.tbl VALUES (42,'fred')").executeUpdate() + conn.prepareStatement("INSERT INTO foo.tbl VALUES (17,'dave')").executeUpdate() + + conn.prepareStatement("CREATE TABLE foo.numbers (onebit BIT(1), tenbits BIT(10), " + + "small SMALLINT, med MEDIUMINT, nor INT, big BIGINT, deci DECIMAL(40,20), flt FLOAT, " + + "dbl DOUBLE)").executeUpdate() + conn.prepareStatement("INSERT INTO foo.numbers VALUES (b'0', b'1000100101', " + + "17, 77777, 123456789, 123456789012345, 123456789012345.123456789012345, " + + "42.75, 1.0000000000000002)").executeUpdate() + + conn.prepareStatement("CREATE TABLE foo.dates (d DATE, t TIME, dt DATETIME, ts TIMESTAMP, " + + "yr YEAR)").executeUpdate() + conn.prepareStatement("INSERT INTO foo.dates VALUES ('1991-11-09', '13:31:24', " + + "'1996-01-01 01:23:45', '2009-02-13 23:31:30', '2001')").executeUpdate() + + // TODO: Test locale conversion for strings. + conn.prepareStatement("CREATE TABLE foo.strings (a CHAR(10), b VARCHAR(10), c TINYTEXT, " + + "d TEXT, e MEDIUMTEXT, f LONGTEXT, g BINARY(4), h VARBINARY(10), i BLOB)" + ).executeUpdate() + conn.prepareStatement("INSERT INTO foo.strings VALUES ('the', 'quick', 'brown', 'fox', 'jumps', 'over', 'the', 'lazy', 'dog')").executeUpdate() + } finally { + conn.close() + } + } + + var db: MySQLDatabase = null + + override def beforeAll() { + // If you load the MySQL driver here, DriverManager will deadlock. The + // MySQL driver gets loaded when its jar gets loaded, unlike the Postgres + // and H2 drivers. + //Class.forName("com.mysql.jdbc.Driver") + + db = new MySQLDatabase() + waitForDatabase(db.ip, 60000) + setupDatabase(db.ip) + ip = db.ip + } + + override def afterAll() { + db.close() + } + + test("Basic test") { + val rdd = TestSQLContext.jdbcRDD(url(ip, "foo"), "tbl") + val rows = rdd.collect + assert(rows.length == 2) + val types = rows(0).toSeq.map(x => x.getClass.toString) + assert(types.length == 2) + assert(types(0).equals("class java.lang.Integer")) + assert(types(1).equals("class java.lang.String")) + } + + test("Numeric types") { + val rdd = TestSQLContext.jdbcRDD(url(ip, "foo"), "numbers") + val rows = rdd.collect + assert(rows.length == 1) + val types = rows(0).toSeq.map(x => x.getClass.toString) + assert(types.length == 9) + println(types(1)) + assert(types(0).equals("class java.lang.Boolean")) + assert(types(1).equals("class java.lang.Long")) + assert(types(2).equals("class java.lang.Integer")) + assert(types(3).equals("class java.lang.Integer")) + assert(types(4).equals("class java.lang.Integer")) + assert(types(5).equals("class java.lang.Long")) + assert(types(6).equals("class java.math.BigDecimal")) + assert(types(7).equals("class java.lang.Double")) + assert(types(8).equals("class java.lang.Double")) + assert(rows(0).getBoolean(0) == false) + assert(rows(0).getLong(1) == 0x225) + assert(rows(0).getInt(2) == 17) + assert(rows(0).getInt(3) == 77777) + assert(rows(0).getInt(4) == 123456789) + assert(rows(0).getLong(5) == 123456789012345L) + val bd = new BigDecimal("123456789012345.12345678901234500000") + assert(rows(0).getAs[BigDecimal](6).equals(bd)) + assert(rows(0).getDouble(7) == 42.75) + assert(rows(0).getDouble(8) == 1.0000000000000002) + } + + test("Date types") { + val rdd = TestSQLContext.jdbcRDD(url(ip, "foo"), "dates") + val rows = rdd.collect + assert(rows.length == 1) + val types = rows(0).toSeq.map(x => x.getClass.toString) + assert(types.length == 5) + assert(types(0).equals("class java.sql.Date")) + assert(types(1).equals("class java.sql.Timestamp")) + assert(types(2).equals("class java.sql.Timestamp")) + assert(types(3).equals("class java.sql.Timestamp")) + assert(types(4).equals("class java.sql.Date")) + assert(rows(0).getAs[Date](0).equals(new Date(91, 10, 9))) + assert(rows(0).getAs[Timestamp](1).equals(new Timestamp(70, 0, 1, 13, 31, 24, 0))) + assert(rows(0).getAs[Timestamp](2).equals(new Timestamp(96, 0, 1, 1, 23, 45, 0))) + assert(rows(0).getAs[Timestamp](3).equals(new Timestamp(109, 1, 13, 23, 31, 30, 0))) + assert(rows(0).getAs[Date](4).equals(new Date(101, 0, 1))) + } + + test("String types") { + val rdd = TestSQLContext.jdbcRDD(url(ip, "foo"), "strings") + val rows = rdd.collect + assert(rows.length == 1) + val types = rows(0).toSeq.map(x => x.getClass.toString) + assert(types.length == 9) + assert(types(0).equals("class java.lang.String")) + assert(types(1).equals("class java.lang.String")) + assert(types(2).equals("class java.lang.String")) + assert(types(3).equals("class java.lang.String")) + assert(types(4).equals("class java.lang.String")) + assert(types(5).equals("class java.lang.String")) + assert(types(6).equals("class [B")) + assert(types(7).equals("class [B")) + assert(types(8).equals("class [B")) + assert(rows(0).getString(0).equals("the")) + assert(rows(0).getString(1).equals("quick")) + assert(rows(0).getString(2).equals("brown")) + assert(rows(0).getString(3).equals("fox")) + assert(rows(0).getString(4).equals("jumps")) + assert(rows(0).getString(5).equals("over")) + assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](6), Array[Byte](116, 104, 101, 0))) + assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](7), Array[Byte](108, 97, 122, 121))) + assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](8), Array[Byte](100, 111, 103))) + } + + test("Basic write test") { + val rdd1 = TestSQLContext.jdbcRDD(url(ip, "foo"), "numbers") + val rdd2 = TestSQLContext.jdbcRDD(url(ip, "foo"), "dates") + val rdd3 = TestSQLContext.jdbcRDD(url(ip, "foo"), "strings") + rdd1.createJDBCTable(url(ip, "foo"), "numberscopy", false) + rdd2.createJDBCTable(url(ip, "foo"), "datescopy", false) + rdd3.createJDBCTable(url(ip, "foo"), "stringscopy", false) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala new file mode 100644 index 0000000000000..c174d7adb7204 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.jdbc + +import java.math.BigDecimal +import org.apache.spark.sql.test._ +import org.scalatest.{FunSuite, BeforeAndAfterAll, Ignore} +import java.sql.DriverManager +import TestSQLContext._ +import com.spotify.docker.client.{DefaultDockerClient, DockerClient} +import com.spotify.docker.client.messages.ContainerConfig + +class PostgresDatabase { + val docker: DockerClient = DockerClientFactory.get() + val containerId = { + println("Pulling postgres") + docker.pull("postgres") + println("Configuring container") + val config = (ContainerConfig.builder().image("postgres") + .env("POSTGRES_PASSWORD=rootpass") + .build()) + println("Creating container") + val id = docker.createContainer(config).id + println("Starting container " + id) + docker.startContainer(id) + id + } + val ip = docker.inspectContainer(containerId).networkSettings.ipAddress + + def close() { + try { + println("Killing container " + containerId) + docker.killContainer(containerId) + println("Removing container " + containerId) + docker.removeContainer(containerId) + println("Closing docker client") + DockerClientFactory.close(docker) + } catch { + case e: Exception => { + println(e) + println("You may need to clean this up manually.") + throw e + } + } + } +} + +@Ignore class PostgresIntegration extends FunSuite with BeforeAndAfterAll { + lazy val db = new PostgresDatabase() + + def url(ip: String) = s"jdbc:postgresql://$ip:5432/postgres?user=postgres&password=rootpass" + + def waitForDatabase(ip: String, maxMillis: Long) { + val before = System.currentTimeMillis() + var lastException: java.sql.SQLException = null + while (true) { + if (System.currentTimeMillis() > before + maxMillis) { + throw new java.sql.SQLException(s"Database not up after $maxMillis ms.", + lastException) + } + try { + val conn = java.sql.DriverManager.getConnection(url(ip)) + conn.close() + println("Database is up.") + return; + } catch { + case e: java.sql.SQLException => { + lastException = e + java.lang.Thread.sleep(250) + } + } + } + } + + def setupDatabase(ip: String) { + val conn = DriverManager.getConnection(url(ip)) + try { + conn.prepareStatement("CREATE DATABASE foo").executeUpdate() + conn.setCatalog("foo") + conn.prepareStatement("CREATE TABLE bar (a text, b integer, c double precision, d bigint, " + + "e bit(1), f bit(10), g bytea, h boolean, i inet, j cidr)").executeUpdate() + conn.prepareStatement("INSERT INTO bar VALUES ('hello', 42, 1.25, 123456789012345, B'0', " + + "B'1000100101', E'\\\\xDEADBEEF', true, '172.16.0.42', '192.168.0.0/16')").executeUpdate() + } finally { + conn.close() + } + } + + override def beforeAll() { + println("Waiting for database to start up.") + waitForDatabase(db.ip, 60000) + println("Setting up database.") + setupDatabase(db.ip) + } + + override def afterAll() { + db.close() + } + + test("Type mapping for various types") { + val rdd = TestSQLContext.jdbcRDD(url(db.ip), "public.bar") + val rows = rdd.collect + assert(rows.length == 1) + val types = rows(0).toSeq.map(x => x.getClass.toString) + assert(types.length == 10) + assert(types(0).equals("class java.lang.String")) + assert(types(1).equals("class java.lang.Integer")) + assert(types(2).equals("class java.lang.Double")) + assert(types(3).equals("class java.lang.Long")) + assert(types(4).equals("class java.lang.Boolean")) + assert(types(5).equals("class [B")) + assert(types(6).equals("class [B")) + assert(types(7).equals("class java.lang.Boolean")) + assert(types(8).equals("class java.lang.String")) + assert(types(9).equals("class java.lang.String")) + assert(rows(0).getString(0).equals("hello")) + assert(rows(0).getInt(1) == 42) + assert(rows(0).getDouble(2) == 1.25) + assert(rows(0).getLong(3) == 123456789012345L) + assert(rows(0).getBoolean(4) == false) + // BIT(10)'s come back as ASCII strings of ten ASCII 0's and 1's... + assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](5), Array[Byte](49,48,48,48,49,48,48,49,48,49))) + assert(java.util.Arrays.equals(rows(0).getAs[Array[Byte]](6), Array[Byte](0xDE.toByte, 0xAD.toByte, 0xBE.toByte, 0xEF.toByte))) + assert(rows(0).getBoolean(7) == true) + assert(rows(0).getString(8) == "172.16.0.42") + assert(rows(0).getString(9) == "192.168.0.0/16") + } + + test("Basic write test") { + val rdd = TestSQLContext.jdbcRDD(url(db.ip), "public.bar") + rdd.createJDBCTable(url(db.ip), "public.barcopy", false) + // Test only that it doesn't bomb out. + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala index e78145f4dda5a..ff91a0eb42049 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal, Predicate, Row} import org.apache.spark.sql.types._ import org.apache.spark.sql.test.TestSQLContext -import org.apache.spark.sql.{DataFrame, QueryTest, SQLConf} +import org.apache.spark.sql.{Column, DataFrame, QueryTest, SQLConf} /** * A test suite that tests Parquet filter2 API based filter pushdown optimization. @@ -51,8 +51,8 @@ class ParquetFilterSuite extends QueryTest with ParquetTest { withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED -> "true") { val query = rdd - .select(output.map(e => new org.apache.spark.sql.Column(e)): _*) - .where(new org.apache.spark.sql.Column(predicate)) + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) val maybeAnalyzedPredicate = query.queryExecution.executedPlan.collect { case plan: ParquetTableScan => plan.columnPruningPred diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index b746942cb1067..5efc3b1e30774 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -72,7 +72,8 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { if (conf.dialect == "sql") { super.sql(substituted) } else if (conf.dialect == "hiveql") { - new DataFrame(this, ddlParser(sqlText, false).getOrElse(HiveQl.parseSql(substituted))) + DataFrame(this, + ddlParser(sqlText, exceptionOnError = false).getOrElse(HiveQl.parseSql(substituted))) } else { sys.error(s"Unsupported SQL dialect: ${conf.dialect}. Try 'sql' or 'hiveql'") } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 1a49f09bd9988..d910ee950904d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -519,6 +519,15 @@ private[hive] case class MetastoreRelation } ) + /** Only compare database and tablename, not alias. */ + override def sameResult(plan: LogicalPlan): Boolean = { + plan match { + case mr: MetastoreRelation => + mr.databaseName == databaseName && mr.tableName == tableName + case _ => false + } + } + val tableDesc = HiveShim.getTableDesc( Class.forName( hiveQlTable.getSerializationLib, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 30a64b48d7951..ab305e1f82a55 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.ExplainCommand -import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DropTable, AnalyzeTable} +import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DropTable, AnalyzeTable, HiveScriptIOSchema} import org.apache.spark.sql.types._ /* Implicit conversions */ @@ -627,29 +627,64 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case Token("TOK_SELEXPR", Token("TOK_TRANSFORM", Token("TOK_EXPLIST", inputExprs) :: - Token("TOK_SERDE", Nil) :: + Token("TOK_SERDE", inputSerdeClause) :: Token("TOK_RECORDWRITER", writerClause) :: // TODO: Need to support other types of (in/out)put Token(script, Nil) :: - Token("TOK_SERDE", serdeClause) :: + Token("TOK_SERDE", outputSerdeClause) :: Token("TOK_RECORDREADER", readerClause) :: - outputClause :: Nil) :: Nil) => - - val output = outputClause match { - case Token("TOK_ALIASLIST", aliases) => - aliases.map { case Token(name, Nil) => AttributeReference(name, StringType)() } - case Token("TOK_TABCOLLIST", attributes) => - attributes.map { case Token("TOK_TABCOL", Token(name, Nil) :: dataType :: Nil) => - AttributeReference(name, nodeToDataType(dataType))() } + outputClause) :: Nil) => + + val (output, schemaLess) = outputClause match { + case Token("TOK_ALIASLIST", aliases) :: Nil => + (aliases.map { case Token(name, Nil) => AttributeReference(name, StringType)() }, + false) + case Token("TOK_TABCOLLIST", attributes) :: Nil => + (attributes.map { case Token("TOK_TABCOL", Token(name, Nil) :: dataType :: Nil) => + AttributeReference(name, nodeToDataType(dataType))() }, false) + case Nil => + (List(AttributeReference("key", StringType)(), + AttributeReference("value", StringType)()), true) } + + def matchSerDe(clause: Seq[ASTNode]) = clause match { + case Token("TOK_SERDEPROPS", propsClause) :: Nil => + val rowFormat = propsClause.map { + case Token(name, Token(value, Nil) :: Nil) => (name, value) + } + (rowFormat, "", Nil) + + case Token("TOK_SERDENAME", Token(serdeClass, Nil) :: Nil) :: Nil => + (Nil, serdeClass, Nil) + + case Token("TOK_SERDENAME", Token(serdeClass, Nil) :: + Token("TOK_TABLEPROPERTIES", + Token("TOK_TABLEPROPLIST", propsClause) :: Nil) :: Nil) :: Nil => + val serdeProps = propsClause.map { + case Token("TOK_TABLEPROPERTY", Token(name, Nil) :: Token(value, Nil) :: Nil) => + (name, value) + } + (Nil, serdeClass, serdeProps) + + case Nil => (Nil, "", Nil) + } + + val (inRowFormat, inSerdeClass, inSerdeProps) = matchSerDe(inputSerdeClause) + val (outRowFormat, outSerdeClass, outSerdeProps) = matchSerDe(outputSerdeClause) + val unescapedScript = BaseSemanticAnalyzer.unescapeSQLString(script) + val schema = HiveScriptIOSchema( + inRowFormat, outRowFormat, + inSerdeClass, outSerdeClass, + inSerdeProps, outSerdeProps, schemaLess) + Some( logical.ScriptTransformation( inputExprs.map(nodeToExpr), unescapedScript, output, - withWhere)) + withWhere, schema)) case _ => None } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index ace9329cd5821..fa997288a2848 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -17,10 +17,12 @@ package org.apache.spark.sql.hive +import org.apache.spark.sql.catalyst.expressions.Row + import scala.collection.JavaConversions._ import org.apache.spark.annotation.Experimental -import org.apache.spark.sql.{Column, DataFrame, SQLContext, Strategy} +import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate @@ -29,7 +31,6 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.{DescribeCommand => RunnableDescribeCommand} import org.apache.spark.sql.execution._ -import org.apache.spark.sql.hive import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.parquet.ParquetRelation import org.apache.spark.sql.sources.CreateTableUsing @@ -56,14 +57,14 @@ private[hive] trait HiveStrategies { @Experimental object ParquetConversion extends Strategy { implicit class LogicalPlanHacks(s: DataFrame) { - def lowerCase = new DataFrame(s.sqlContext, s.logicalPlan) + def lowerCase = DataFrame(s.sqlContext, s.logicalPlan) def addPartitioningAttributes(attrs: Seq[Attribute]) = { // Don't add the partitioning key if its already present in the data. if (attrs.map(_.name).toSet.subsetOf(s.logicalPlan.output.map(_.name).toSet)) { s } else { - new DataFrame( + DataFrame( s.sqlContext, s.logicalPlan transform { case p: ParquetRelation => p.copy(partitioningAttributes = attrs) @@ -96,13 +97,13 @@ private[hive] trait HiveStrategies { // We are going to throw the predicates and projection back at the whole optimization // sequence so lets unresolve all the attributes, allowing them to be rebound to the // matching parquet attributes. - val unresolvedOtherPredicates = new Column(otherPredicates.map(_ transform { + val unresolvedOtherPredicates = Column(otherPredicates.map(_ transform { case a: AttributeReference => UnresolvedAttribute(a.name) }).reduceOption(And).getOrElse(Literal(true))) val unresolvedProjection: Seq[Column] = projectList.map(_ transform { case a: AttributeReference => UnresolvedAttribute(a.name) - }).map(new Column(_)) + }).map(Column(_)) try { if (relation.hiveQlTable.isPartitioned) { @@ -166,8 +167,8 @@ private[hive] trait HiveStrategies { object Scripts extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.ScriptTransformation(input, script, output, child) => - ScriptTransformation(input, script, output, planLater(child))(hiveContext) :: Nil + case logical.ScriptTransformation(input, script, output, child, schema: HiveScriptIOSchema) => + ScriptTransformation(input, script, output, planLater(child), schema)(hiveContext) :: Nil case _ => Nil } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index 0c8f676e9c5c8..c54fbb6e24690 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -18,11 +18,26 @@ package org.apache.spark.sql.hive.execution import java.io.{BufferedReader, InputStreamReader} +import java.io.{DataInputStream, DataOutputStream, EOFException} +import java.util.Properties + +import org.apache.hadoop.hive.serde.serdeConstants +import org.apache.hadoop.hive.serde2.AbstractSerDe +import org.apache.hadoop.hive.serde2.Serializer +import org.apache.hadoop.hive.serde2.Deserializer +import org.apache.hadoop.hive.serde2.objectinspector._ +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.ScriptInputOutputSchema import org.apache.spark.sql.execution._ -import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.hive.{HiveContext, HiveInspectors} +import org.apache.spark.sql.hive.HiveShim._ +import org.apache.spark.util.Utils + /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -40,7 +55,8 @@ case class ScriptTransformation( input: Seq[Expression], script: String, output: Seq[Attribute], - child: SparkPlan)(@transient sc: HiveContext) + child: SparkPlan, + ioschema: HiveScriptIOSchema)(@transient sc: HiveContext) extends UnaryNode { override def otherCopyArgs = sc :: Nil @@ -53,28 +69,202 @@ case class ScriptTransformation( val inputStream = proc.getInputStream val outputStream = proc.getOutputStream val reader = new BufferedReader(new InputStreamReader(inputStream)) + + val (outputSerde, outputSoi) = ioschema.initOutputSerDe(output) + + val iterator: Iterator[Row] = new Iterator[Row] with HiveInspectors { + var cacheRow: Row = null + var curLine: String = null + var eof: Boolean = false + + override def hasNext: Boolean = { + if (outputSerde == null) { + if (curLine == null) { + curLine = reader.readLine() + curLine != null + } else { + true + } + } else { + !eof + } + } + + def deserialize(): Row = { + if (cacheRow != null) return cacheRow + + val mutableRow = new SpecificMutableRow(output.map(_.dataType)) + try { + val dataInputStream = new DataInputStream(inputStream) + val writable = outputSerde.getSerializedClass().newInstance + writable.readFields(dataInputStream) + + val raw = outputSerde.deserialize(writable) + val dataList = outputSoi.getStructFieldsDataAsList(raw) + val fieldList = outputSoi.getAllStructFieldRefs() + + var i = 0 + dataList.foreach( element => { + if (element == null) { + mutableRow.setNullAt(i) + } else { + mutableRow(i) = unwrap(element, fieldList(i).getFieldObjectInspector) + } + i += 1 + }) + return mutableRow + } catch { + case e: EOFException => + eof = true + return null + } + } - // TODO: This should be exposed as an iterator instead of reading in all the data at once. - val outputLines = collection.mutable.ArrayBuffer[Row]() - val readerThread = new Thread("Transform OutputReader") { - override def run() { - var curLine = reader.readLine() - while (curLine != null) { - // TODO: Use SerDe - outputLines += new GenericRow(curLine.split("\t").asInstanceOf[Array[Any]]) + override def next(): Row = { + if (!hasNext) { + throw new NoSuchElementException + } + + if (outputSerde == null) { + val prevLine = curLine curLine = reader.readLine() + + if (!ioschema.schemaLess) { + new GenericRow( + prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD")) + .asInstanceOf[Array[Any]]) + } else { + new GenericRow( + prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD"), 2) + .asInstanceOf[Array[Any]]) + } + } else { + val ret = deserialize() + if (!eof) { + cacheRow = null + cacheRow = deserialize() + } + ret } } } - readerThread.start() + + val (inputSerde, inputSoi) = ioschema.initInputSerDe(input) + val dataOutputStream = new DataOutputStream(outputStream) val outputProjection = new InterpretedProjection(input, child.output) + iter .map(outputProjection) - // TODO: Use SerDe - .map(_.mkString("", "\t", "\n").getBytes("utf-8")).foreach(outputStream.write) + .foreach { row => + if (inputSerde == null) { + val data = row.mkString("", ioschema.inputRowFormatMap("TOK_TABLEROWFORMATFIELD"), + ioschema.inputRowFormatMap("TOK_TABLEROWFORMATLINES")).getBytes("utf-8") + + outputStream.write(data) + } else { + val writable = inputSerde.serialize(row.asInstanceOf[GenericRow].values, inputSoi) + prepareWritable(writable).write(dataOutputStream) + } + } outputStream.close() - readerThread.join() - outputLines.toIterator + iterator + } + } +} + +/** + * The wrapper class of Hive input and output schema properties + */ +case class HiveScriptIOSchema ( + inputRowFormat: Seq[(String, String)], + outputRowFormat: Seq[(String, String)], + inputSerdeClass: String, + outputSerdeClass: String, + inputSerdeProps: Seq[(String, String)], + outputSerdeProps: Seq[(String, String)], + schemaLess: Boolean) extends ScriptInputOutputSchema with HiveInspectors { + + val defaultFormat = Map(("TOK_TABLEROWFORMATFIELD", "\t"), + ("TOK_TABLEROWFORMATLINES", "\n")) + + val inputRowFormatMap = inputRowFormat.toMap.withDefault((k) => defaultFormat(k)) + val outputRowFormatMap = outputRowFormat.toMap.withDefault((k) => defaultFormat(k)) + + + def initInputSerDe(input: Seq[Expression]): (AbstractSerDe, ObjectInspector) = { + val (columns, columnTypes) = parseAttrs(input) + val serde = initSerDe(inputSerdeClass, columns, columnTypes, inputSerdeProps) + (serde, initInputSoi(serde, columns, columnTypes)) + } + + def initOutputSerDe(output: Seq[Attribute]): (AbstractSerDe, StructObjectInspector) = { + val (columns, columnTypes) = parseAttrs(output) + val serde = initSerDe(outputSerdeClass, columns, columnTypes, outputSerdeProps) + (serde, initOutputputSoi(serde)) + } + + def parseAttrs(attrs: Seq[Expression]): (Seq[String], Seq[DataType]) = { + + val columns = attrs.map { + case aref: AttributeReference => aref.name + case e: NamedExpression => e.name + case _ => null + } + + val columnTypes = attrs.map { + case aref: AttributeReference => aref.dataType + case e: NamedExpression => e.dataType + case _ => null + } + + (columns, columnTypes) + } + + def initSerDe(serdeClassName: String, columns: Seq[String], + columnTypes: Seq[DataType], serdeProps: Seq[(String, String)]): AbstractSerDe = { + + val serde: AbstractSerDe = if (serdeClassName != "") { + val trimed_class = serdeClassName.split("'")(1) + Utils.classForName(trimed_class) + .newInstance.asInstanceOf[AbstractSerDe] + } else { + null + } + + if (serde != null) { + val columnTypesNames = columnTypes.map(_.toTypeInfo.getTypeName()).mkString(",") + + var propsMap = serdeProps.map(kv => { + (kv._1.split("'")(1), kv._2.split("'")(1)) + }).toMap + (serdeConstants.LIST_COLUMNS -> columns.mkString(",")) + propsMap = propsMap + (serdeConstants.LIST_COLUMN_TYPES -> columnTypesNames) + + val properties = new Properties() + properties.putAll(propsMap) + serde.initialize(null, properties) + } + + serde + } + + def initInputSoi(inputSerde: AbstractSerDe, columns: Seq[String], columnTypes: Seq[DataType]) + : ObjectInspector = { + + if (inputSerde != null) { + val fieldObjectInspectors = columnTypes.map(toInspector(_)) + ObjectInspectorFactory + .getStandardStructObjectInspector(columns, fieldObjectInspectors) + .asInstanceOf[ObjectInspector] + } else { + null + } + } + + def initOutputputSoi(outputSerde: AbstractSerDe): StructObjectInspector = { + if (outputSerde != null) { + outputSerde.getObjectInspector().asInstanceOf[StructObjectInspector] + } else { + null } } } diff --git a/sql/hive/src/test/resources/golden/schema-less transform-0-d5738de14dd6e29da712ec3318f4118f b/sql/hive/src/test/resources/golden/schema-less transform-0-d5738de14dd6e29da712ec3318f4118f new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/sql/hive/src/test/resources/golden/schema-less transform-0-d5738de14dd6e29da712ec3318f4118f @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/schema-less transform-1-49624ef4e2c3cc2040c06660b926219b b/sql/hive/src/test/resources/golden/schema-less transform-1-49624ef4e2c3cc2040c06660b926219b new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/sql/hive/src/test/resources/golden/schema-less transform-1-49624ef4e2c3cc2040c06660b926219b @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/transform with SerDe-0-cdc393f3914c879787efe523f692b1e0 b/sql/hive/src/test/resources/golden/transform with SerDe-0-cdc393f3914c879787efe523f692b1e0 new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform with SerDe-0-cdc393f3914c879787efe523f692b1e0 @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/transform with SerDe3-0-58a8b7eb07a949bc44dccb723222957f b/sql/hive/src/test/resources/golden/transform with SerDe3-0-58a8b7eb07a949bc44dccb723222957f new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform with SerDe3-0-58a8b7eb07a949bc44dccb723222957f @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/transform with SerDe4-0-ba9ad2499a7408cb350c7abafaf9ea97 b/sql/hive/src/test/resources/golden/transform with SerDe4-0-ba9ad2499a7408cb350c7abafaf9ea97 new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform with SerDe4-0-ba9ad2499a7408cb350c7abafaf9ea97 @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/transform with custom field delimiter-0-703cca3c02ced422feb11dc13b744484 b/sql/hive/src/test/resources/golden/transform with custom field delimiter-0-703cca3c02ced422feb11dc13b744484 new file mode 100644 index 0000000000000..e34118512c1d7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform with custom field delimiter-0-703cca3c02ced422feb11dc13b744484 @@ -0,0 +1,500 @@ +238 +86 +311 +27 +165 +409 +255 +278 +98 +484 +265 +193 +401 +150 +273 +224 +369 +66 +128 +213 +146 +406 +429 +374 +152 +469 +145 +495 +37 +327 +281 +277 +209 +15 +82 +403 +166 +417 +430 +252 +292 +219 +287 +153 +193 +338 +446 +459 +394 +237 +482 +174 +413 +494 +207 +199 +466 +208 +174 +399 +396 +247 +417 +489 +162 +377 +397 +309 +365 +266 +439 +342 +367 +325 +167 +195 +475 +17 +113 +155 +203 +339 +0 +455 +128 +311 +316 +57 +302 +205 +149 +438 +345 +129 +170 +20 +489 +157 +378 +221 +92 +111 +47 +72 +4 +280 +35 +427 +277 +208 +356 +399 +169 +382 +498 +125 +386 +437 +469 +192 +286 +187 +176 +54 +459 +51 +138 +103 +239 +213 +216 +430 +278 +176 +289 +221 +65 +318 +332 +311 +275 +137 +241 +83 +333 +180 +284 +12 +230 +181 +67 +260 +404 +384 +489 +353 +373 +272 +138 +217 +84 +348 +466 +58 +8 +411 +230 +208 +348 +24 +463 +431 +179 +172 +42 +129 +158 +119 +496 +0 +322 +197 +468 +393 +454 +100 +298 +199 +191 +418 +96 +26 +165 +327 +230 +205 +120 +131 +51 +404 +43 +436 +156 +469 +468 +308 +95 +196 +288 +481 +457 +98 +282 +197 +187 +318 +318 +409 +470 +137 +369 +316 +169 +413 +85 +77 +0 +490 +87 +364 +179 +118 +134 +395 +282 +138 +238 +419 +15 +118 +72 +90 +307 +19 +435 +10 +277 +273 +306 +224 +309 +389 +327 +242 +369 +392 +272 +331 +401 +242 +452 +177 +226 +5 +497 +402 +396 +317 +395 +58 +35 +336 +95 +11 +168 +34 +229 +233 +143 +472 +322 +498 +160 +195 +42 +321 +430 +119 +489 +458 +78 +76 +41 +223 +492 +149 +449 +218 +228 +138 +453 +30 +209 +64 +468 +76 +74 +342 +69 +230 +33 +368 +103 +296 +113 +216 +367 +344 +167 +274 +219 +239 +485 +116 +223 +256 +263 +70 +487 +480 +401 +288 +191 +5 +244 +438 +128 +467 +432 +202 +316 +229 +469 +463 +280 +2 +35 +283 +331 +235 +80 +44 +193 +321 +335 +104 +466 +366 +175 +403 +483 +53 +105 +257 +406 +409 +190 +406 +401 +114 +258 +90 +203 +262 +348 +424 +12 +396 +201 +217 +164 +431 +454 +478 +298 +125 +431 +164 +424 +187 +382 +5 +70 +397 +480 +291 +24 +351 +255 +104 +70 +163 +438 +119 +414 +200 +491 +237 +439 +360 +248 +479 +305 +417 +199 +444 +120 +429 +169 +443 +323 +325 +277 +230 +478 +178 +468 +310 +317 +333 +493 +460 +207 +249 +265 +480 +83 +136 +353 +172 +214 +462 +233 +406 +133 +175 +189 +454 +375 +401 +421 +407 +384 +256 +26 +134 +67 +384 +379 +18 +462 +492 +100 +298 +9 +341 +498 +146 +458 +362 +186 +285 +348 +167 +18 +273 +183 +281 +344 +97 +469 +315 +84 +28 +37 +448 +152 +348 +307 +194 +414 +477 +222 +126 +90 +169 +403 +400 +200 +97 diff --git a/sql/hive/src/test/resources/golden/transform with custom field delimiter-0-82639dda9ba42df817466dffe2929174 b/sql/hive/src/test/resources/golden/transform with custom field delimiter-0-82639dda9ba42df817466dffe2929174 new file mode 100644 index 0000000000000..e34118512c1d7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform with custom field delimiter-0-82639dda9ba42df817466dffe2929174 @@ -0,0 +1,500 @@ +238 +86 +311 +27 +165 +409 +255 +278 +98 +484 +265 +193 +401 +150 +273 +224 +369 +66 +128 +213 +146 +406 +429 +374 +152 +469 +145 +495 +37 +327 +281 +277 +209 +15 +82 +403 +166 +417 +430 +252 +292 +219 +287 +153 +193 +338 +446 +459 +394 +237 +482 +174 +413 +494 +207 +199 +466 +208 +174 +399 +396 +247 +417 +489 +162 +377 +397 +309 +365 +266 +439 +342 +367 +325 +167 +195 +475 +17 +113 +155 +203 +339 +0 +455 +128 +311 +316 +57 +302 +205 +149 +438 +345 +129 +170 +20 +489 +157 +378 +221 +92 +111 +47 +72 +4 +280 +35 +427 +277 +208 +356 +399 +169 +382 +498 +125 +386 +437 +469 +192 +286 +187 +176 +54 +459 +51 +138 +103 +239 +213 +216 +430 +278 +176 +289 +221 +65 +318 +332 +311 +275 +137 +241 +83 +333 +180 +284 +12 +230 +181 +67 +260 +404 +384 +489 +353 +373 +272 +138 +217 +84 +348 +466 +58 +8 +411 +230 +208 +348 +24 +463 +431 +179 +172 +42 +129 +158 +119 +496 +0 +322 +197 +468 +393 +454 +100 +298 +199 +191 +418 +96 +26 +165 +327 +230 +205 +120 +131 +51 +404 +43 +436 +156 +469 +468 +308 +95 +196 +288 +481 +457 +98 +282 +197 +187 +318 +318 +409 +470 +137 +369 +316 +169 +413 +85 +77 +0 +490 +87 +364 +179 +118 +134 +395 +282 +138 +238 +419 +15 +118 +72 +90 +307 +19 +435 +10 +277 +273 +306 +224 +309 +389 +327 +242 +369 +392 +272 +331 +401 +242 +452 +177 +226 +5 +497 +402 +396 +317 +395 +58 +35 +336 +95 +11 +168 +34 +229 +233 +143 +472 +322 +498 +160 +195 +42 +321 +430 +119 +489 +458 +78 +76 +41 +223 +492 +149 +449 +218 +228 +138 +453 +30 +209 +64 +468 +76 +74 +342 +69 +230 +33 +368 +103 +296 +113 +216 +367 +344 +167 +274 +219 +239 +485 +116 +223 +256 +263 +70 +487 +480 +401 +288 +191 +5 +244 +438 +128 +467 +432 +202 +316 +229 +469 +463 +280 +2 +35 +283 +331 +235 +80 +44 +193 +321 +335 +104 +466 +366 +175 +403 +483 +53 +105 +257 +406 +409 +190 +406 +401 +114 +258 +90 +203 +262 +348 +424 +12 +396 +201 +217 +164 +431 +454 +478 +298 +125 +431 +164 +424 +187 +382 +5 +70 +397 +480 +291 +24 +351 +255 +104 +70 +163 +438 +119 +414 +200 +491 +237 +439 +360 +248 +479 +305 +417 +199 +444 +120 +429 +169 +443 +323 +325 +277 +230 +478 +178 +468 +310 +317 +333 +493 +460 +207 +249 +265 +480 +83 +136 +353 +172 +214 +462 +233 +406 +133 +175 +189 +454 +375 +401 +421 +407 +384 +256 +26 +134 +67 +384 +379 +18 +462 +492 +100 +298 +9 +341 +498 +146 +458 +362 +186 +285 +348 +167 +18 +273 +183 +281 +344 +97 +469 +315 +84 +28 +37 +448 +152 +348 +307 +194 +414 +477 +222 +126 +90 +169 +403 +400 +200 +97 diff --git a/sql/hive/src/test/resources/golden/transform with custom field delimiter2-0-e8713b21483e1efb78ee90b61530479b b/sql/hive/src/test/resources/golden/transform with custom field delimiter2-0-e8713b21483e1efb78ee90b61530479b new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform with custom field delimiter2-0-e8713b21483e1efb78ee90b61530479b @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/transform with custom field delimiter2-0-e8d2b2e60551f69bfb44e555f5cff064 b/sql/hive/src/test/resources/golden/transform with custom field delimiter2-0-e8d2b2e60551f69bfb44e555f5cff064 new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform with custom field delimiter2-0-e8d2b2e60551f69bfb44e555f5cff064 @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/transform with custom field delimiter3-0-d4f4f471819345e9ce1964e281ea5289 b/sql/hive/src/test/resources/golden/transform with custom field delimiter3-0-d4f4f471819345e9ce1964e281ea5289 new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform with custom field delimiter3-0-d4f4f471819345e9ce1964e281ea5289 @@ -0,0 +1,500 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +302 val_302 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 61e5117feab10..7c8b5205e239e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -64,6 +64,12 @@ class CachedTableSuite extends QueryTest { sql("SELECT * FROM src"), preCacheResults) + assertCached(sql("SELECT * FROM src s")) + + checkAnswer( + sql("SELECT * FROM src s"), + preCacheResults) + uncacheTable("src") assertCached(sql("SELECT * FROM src"), 0) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 60619f5d99578..4c53b10ba96e9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -63,7 +63,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { sql("SHOW TABLES") } } - + createQueryTest("! operator", """ |SELECT a FROM ( @@ -329,6 +329,80 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { createQueryTest("transform", "SELECT TRANSFORM (key) USING 'cat' AS (tKey) FROM src") + createQueryTest("schema-less transform", + """ + |SELECT TRANSFORM (key, value) USING 'cat' FROM src; + |SELECT TRANSFORM (*) USING 'cat' FROM src; + """.stripMargin) + + val delimiter = "'\t'" + + createQueryTest("transform with custom field delimiter", + s""" + |SELECT TRANSFORM (key) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} + |USING 'cat' AS (tKey) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} FROM src; + """.stripMargin.replaceAll("\n", " ")) + + createQueryTest("transform with custom field delimiter2", + s""" + |SELECT TRANSFORM (key, value) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} + |USING 'cat' ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} FROM src; + """.stripMargin.replaceAll("\n", " ")) + + createQueryTest("transform with custom field delimiter3", + s""" + |SELECT TRANSFORM (*) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} + |USING 'cat' ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} FROM src; + """.stripMargin.replaceAll("\n", " ")) + + createQueryTest("transform with SerDe", + """ + |SELECT TRANSFORM (key, value) ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' + |USING 'cat' AS (tKey, tValue) ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' FROM src; + """.stripMargin.replaceAll("\n", " ")) + + test("transform with SerDe2") { + + sql("CREATE TABLE small_src(key INT, value STRING)") + sql("INSERT OVERWRITE TABLE small_src SELECT key, value FROM src LIMIT 10") + + val expected = sql("SELECT key FROM small_src").collect().head + val res = sql( + """ + |SELECT TRANSFORM (key) ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.avro.AvroSerDe' + |WITH SERDEPROPERTIES ('avro.schema.literal'='{"namespace": + |"testing.hive.avro.serde","name": "src","type": "record","fields": + |[{"name":"key","type":"int"}]}') USING 'cat' AS (tKey INT) ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.avro.AvroSerDe' WITH SERDEPROPERTIES + |('avro.schema.literal'='{"namespace": "testing.hive.avro.serde","name": + |"src","type": "record","fields": [{"name":"key","type":"int"}]}') + |FROM small_src + """.stripMargin.replaceAll("\n", " ")).collect().head + + assert(expected(0) === res(0)) + } + + createQueryTest("transform with SerDe3", + """ + |SELECT TRANSFORM (*) ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES + |('serialization.last.column.takes.rest'='true') USING 'cat' AS (tKey, tValue) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' + |WITH SERDEPROPERTIES ('serialization.last.column.takes.rest'='true') FROM src; + """.stripMargin.replaceAll("\n", " ")) + + createQueryTest("transform with SerDe4", + """ + |SELECT TRANSFORM (*) ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES + |('serialization.last.column.takes.rest'='true') USING 'cat' ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES + |('serialization.last.column.takes.rest'='true') FROM src; + """.stripMargin.replaceAll("\n", " ")) + createQueryTest("LIKE", "SELECT * FROM src WHERE value LIKE '%1%'") diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala index c0b7741bc3e53..254919e8f6fdc 100644 --- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala +++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala @@ -38,7 +38,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, Primitive import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory import org.apache.hadoop.hive.serde2.objectinspector.primitive.{HiveDecimalObjectInspector, PrimitiveObjectInspectorFactory} import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfo, TypeInfoFactory} -import org.apache.hadoop.io.NullWritable +import org.apache.hadoop.io.{NullWritable, Writable} import org.apache.hadoop.mapred.InputFormat import org.apache.spark.sql.types.{Decimal, DecimalType} @@ -241,6 +241,10 @@ private[hive] object HiveShim { Decimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue()) } } + + def prepareWritable(w: Writable): Writable = { + w + } } class ShimFileSinkDesc(var dir: String, var tableInfo: TableDesc, var compressed: Boolean) diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala index c04cda7bf1537..45ca59ae56a38 100644 --- a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala +++ b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala @@ -19,13 +19,14 @@ package org.apache.spark.sql.hive import java.util.{ArrayList => JArrayList} import java.util.Properties +import java.rmi.server.UID import scala.collection.JavaConversions._ import scala.language.implicitConversions import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hadoop.io.NullWritable +import org.apache.hadoop.io.{NullWritable, Writable} import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.hive.common.StatsSetupConst import org.apache.hadoop.hive.common.`type`.{HiveDecimal} @@ -39,6 +40,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.{HiveDecimalObjec import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, ObjectInspector} import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils} import org.apache.hadoop.hive.serde2.{io => hiveIo} +import org.apache.hadoop.hive.serde2.avro.AvroGenericRecordWritable import org.apache.hadoop.{io => hadoopIo} import org.apache.spark.Logging @@ -395,10 +397,23 @@ private[hive] object HiveShim { Decimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue(), hdoi.precision(), hdoi.scale()) } } + + /* + * Bug introduced in hive-0.13. AvroGenericRecordWritable has a member recordReaderID that + * is needed to initialize before serialization. + */ + def prepareWritable(w: Writable): Writable = { + w match { + case w: AvroGenericRecordWritable => + w.setRecordReaderID(new UID()) + case _ => + } + w + } } /* - * Bug introdiced in hive-0.13. FileSinkDesc is serilizable, but its member path is not. + * Bug introduced in hive-0.13. FileSinkDesc is serilizable, but its member path is not. * Fix it through wrapper. */ class ShimFileSinkDesc(var dir: String, var tableInfo: TableDesc, var compressed: Boolean) diff --git a/streaming/pom.xml b/streaming/pom.xml index d032491e2ff83..1e92ba686a57d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -40,6 +40,8 @@ spark-core_${scala.binary.version} ${project.version} + + com.google.guava guava @@ -48,6 +50,24 @@ org.eclipse.jetty jetty-server + + org.eclipse.jetty + jetty-plus + + + org.eclipse.jetty + jetty-util + + + org.eclipse.jetty + jetty-http + + + org.eclipse.jetty + jetty-servlet + + + org.scala-lang scala-library diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index 716cf2c7f32fc..7d29ed88cfcb4 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -67,8 +67,12 @@ private[streaming] class ReceiverSupervisorImpl( private val trackerActor = { val ip = env.conf.get("spark.driver.host", "localhost") val port = env.conf.getInt("spark.driver.port", 7077) - val url = "akka.tcp://%s@%s:%s/user/ReceiverTracker".format( - SparkEnv.driverActorSystemName, ip, port) + val url = AkkaUtils.address( + AkkaUtils.protocol(env.actorSystem), + SparkEnv.driverActorSystemName, + ip, + port, + "ReceiverTracker") env.actorSystem.actorSelection(url) } diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index d4c40745658c2..2df8cf6a8a3df 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -1769,7 +1769,7 @@ public Iterable call(InputStream in) throws IOException { @SuppressWarnings("unchecked") @Test public void testTextFileStream() throws IOException { - File testDir = Utils.createTempDir(System.getProperty("java.io.tmpdir")); + File testDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark"); List> expected = fileTestPrepare(testDir); JavaDStream input = ssc.textFileStream(testDir.toString()); @@ -1782,7 +1782,7 @@ public void testTextFileStream() throws IOException { @SuppressWarnings("unchecked") @Test public void testFileStream() throws IOException { - File testDir = Utils.createTempDir(System.getProperty("java.io.tmpdir")); + File testDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark"); List> expected = fileTestPrepare(testDir); JavaPairInputDStream inputStream = ssc.fileStream( diff --git a/yarn/pom.xml b/yarn/pom.xml index 7595549e4b6dc..65344aa8738e0 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -58,6 +58,34 @@ org.apache.hadoop hadoop-client + + + + com.google.guava + guava + + + org.eclipse.jetty + jetty-server + + + org.eclipse.jetty + jetty-plus + + + org.eclipse.jetty + jetty-util + + + org.eclipse.jetty + jetty-http + + + org.eclipse.jetty + jetty-servlet + + + org.apache.hadoop hadoop-yarn-server-tests diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index d3e327b2497b7..37e98e01fddf7 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -34,7 +34,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkEnv} import org.apache.spark.SparkException -import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.{PythonRunner, SparkHadoopUtil} import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.scheduler.cluster.YarnSchedulerBackend import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -135,7 +135,7 @@ private[spark] class ApplicationMaster( .get().addShutdownHook(cleanupHook, ApplicationMaster.SHUTDOWN_HOOK_PRIORITY) // Call this to force generation of secret so it gets populated into the - // Hadoop UGI. This has to happen before the startUserClass which does a + // Hadoop UGI. This has to happen before the startUserApplication which does a // doAs in order for the credentials to be passed on to the executor containers. val securityMgr = new SecurityManager(sparkConf) @@ -244,7 +244,9 @@ private[spark] class ApplicationMaster( host: String, port: String, isDriver: Boolean): Unit = { - val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + + val driverUrl = AkkaUtils.address( + AkkaUtils.protocol(actorSystem), SparkEnv.driverActorSystemName, host, port, @@ -254,7 +256,7 @@ private[spark] class ApplicationMaster( private def runDriver(securityMgr: SecurityManager): Unit = { addAmIpFilter() - userClassThread = startUserClass() + userClassThread = startUserApplication() // This a bit hacky, but we need to wait until the spark.driver.port property has // been set by the Thread executing the user class. @@ -448,9 +450,13 @@ private[spark] class ApplicationMaster( * * Returns the user thread that was started. */ - private def startUserClass(): Thread = { - logInfo("Starting the user JAR in a separate Thread") + private def startUserApplication(): Thread = { + logInfo("Starting the user application in a separate Thread") System.setProperty("spark.executor.instances", args.numExecutors.toString) + if (args.primaryPyFile != null && args.primaryPyFile.endsWith(".py")) { + System.setProperty("spark.submit.pyFiles", + PythonRunner.formatPaths(args.pyFiles).mkString(",")) + } val mainMethod = Class.forName(args.userClass, false, Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]]) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala index d76a63276d752..e1a992af3aae7 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -24,6 +24,8 @@ import collection.mutable.ArrayBuffer class ApplicationMasterArguments(val args: Array[String]) { var userJar: String = null var userClass: String = null + var primaryPyFile: String = null + var pyFiles: String = null var userArgs: Seq[String] = Seq[String]() var executorMemory = 1024 var executorCores = 1 @@ -48,6 +50,14 @@ class ApplicationMasterArguments(val args: Array[String]) { userClass = value args = tail + case ("--primary-py-file") :: value :: tail => + primaryPyFile = value + args = tail + + case ("--py-files") :: value :: tail => + pyFiles = value + args = tail + case ("--args" | "--arg") :: value :: tail => userArgsBuffer += value args = tail @@ -81,6 +91,9 @@ class ApplicationMasterArguments(val args: Array[String]) { |Options: | --jar JAR_PATH Path to your application's JAR file | --class CLASS_NAME Name of your application's main class + | --primary-py-file A main Python file + | --py-files PY_FILES Comma-separated list of .zip, .egg, or .py files to + | place on the PYTHONPATH for Python apps. | --args ARGS Arguments to be passed to your application's main class. | Multiple invocations are possible, each will be passed in order. | --num-executors NUM Number of executors to start (Default: 2) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 1a18e6509ef26..91e8574e94e2f 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -21,7 +21,7 @@ import java.net.{InetAddress, UnknownHostException, URI, URISyntaxException} import java.nio.ByteBuffer import scala.collection.JavaConversions._ -import scala.collection.mutable.{HashMap, ListBuffer, Map} +import scala.collection.mutable.{ArrayBuffer, HashMap, ListBuffer, Map} import scala.util.{Try, Success, Failure} import com.google.common.base.Objects @@ -477,17 +477,32 @@ private[spark] class Client( } else { Nil } + val primaryPyFile = + if (args.primaryPyFile != null) { + Seq("--primary-py-file", args.primaryPyFile) + } else { + Nil + } + val pyFiles = + if (args.pyFiles != null) { + Seq("--py-files", args.pyFiles) + } else { + Nil + } val amClass = if (isClusterMode) { Class.forName("org.apache.spark.deploy.yarn.ApplicationMaster").getName } else { Class.forName("org.apache.spark.deploy.yarn.ExecutorLauncher").getName } + if (args.primaryPyFile != null && args.primaryPyFile.endsWith(".py")) { + args.userArgs = ArrayBuffer(args.primaryPyFile, args.pyFiles) ++ args.userArgs + } val userArgs = args.userArgs.flatMap { arg => Seq("--arg", YarnSparkHadoopUtil.escapeForShell(arg)) } val amArgs = - Seq(amClass) ++ userClass ++ userJar ++ userArgs ++ + Seq(amClass) ++ userClass ++ userJar ++ primaryPyFile ++ pyFiles ++ userArgs ++ Seq( "--executor-memory", args.executorMemory.toString + "m", "--executor-cores", args.executorCores.toString, diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index f96b245512271..3bc7eb1abf341 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -30,7 +30,9 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) var archives: String = null var userJar: String = null var userClass: String = null - var userArgs: Seq[String] = Seq[String]() + var pyFiles: String = null + var primaryPyFile: String = null + var userArgs: ArrayBuffer[String] = new ArrayBuffer[String]() var executorMemory = 1024 // MB var executorCores = 1 var numExecutors = DEFAULT_NUMBER_EXECUTORS @@ -75,14 +77,23 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) .orElse(sparkConf.getOption("spark.yarn.dist.archives").map(p => Utils.resolveURIs(p))) .orElse(sys.env.get("SPARK_YARN_DIST_ARCHIVES")) .orNull - // If dynamic allocation is enabled, start at the max number of executors + // If dynamic allocation is enabled, start at the configured initial number of executors. + // Default to minExecutors if no initialExecutors is set. if (isDynamicAllocationEnabled) { + val minExecutorsConf = "spark.dynamicAllocation.minExecutors" + val initialExecutorsConf = "spark.dynamicAllocation.initialExecutors" val maxExecutorsConf = "spark.dynamicAllocation.maxExecutors" - if (!sparkConf.contains(maxExecutorsConf)) { + val minNumExecutors = sparkConf.getInt(minExecutorsConf, 0) + val initialNumExecutors = sparkConf.getInt(initialExecutorsConf, minNumExecutors) + val maxNumExecutors = sparkConf.getInt(maxExecutorsConf, Integer.MAX_VALUE) + + // If defined, initial executors must be between min and max + if (initialNumExecutors < minNumExecutors || initialNumExecutors > maxNumExecutors) { throw new IllegalArgumentException( - s"$maxExecutorsConf must be set if dynamic allocation is enabled!") + s"$initialExecutorsConf must be between $minExecutorsConf and $maxNumExecutors!") } - numExecutors = sparkConf.get(maxExecutorsConf).toInt + + numExecutors = initialNumExecutors } } @@ -123,7 +134,6 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) } private def parseArgs(inputArgs: List[String]): Unit = { - val userArgsBuffer = new ArrayBuffer[String]() var args = inputArgs while (!args.isEmpty) { @@ -136,11 +146,15 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) userClass = value args = tail + case ("--primary-py-file") :: value :: tail => + primaryPyFile = value + args = tail + case ("--args" | "--arg") :: value :: tail => if (args(0) == "--args") { println("--args is deprecated. Use --arg instead.") } - userArgsBuffer += value + userArgs += value args = tail case ("--master-class" | "--am-class") :: value :: tail => @@ -196,6 +210,10 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) addJars = value args = tail + case ("--py-files") :: value :: tail => + pyFiles = value + args = tail + case ("--files") :: value :: tail => files = value args = tail @@ -210,8 +228,6 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) throw new IllegalArgumentException(getUsageMessage(args)) } } - - userArgs = userArgsBuffer.readOnly } private def getUsageMessage(unknownParam: List[String] = null): String = { @@ -223,6 +239,7 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) | --jar JAR_PATH Path to your application's JAR file (required in yarn-cluster | mode) | --class CLASS_NAME Name of your application's main class (required) + | --primary-py-file A main Python file | --arg ARG Argument to be passed to your application's main class. | Multiple invocations are possible, each will be passed in order. | --num-executors NUM Number of executors to start (Default: 2) @@ -235,6 +252,8 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) | 'default') | --addJars jars Comma separated list of local jars that want SparkContext.addJar | to work with. + | --py-files PY_FILES Comma-separated list of .zip, .egg, or .py files to + | place on the PYTHONPATH for Python apps. | --files files Comma separated list of files to be distributed with the job. | --archives archives Comma separated list of archives to be distributed with the job. """.stripMargin diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 3849586c6111e..0dbb6154b3039 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -34,9 +34,10 @@ import org.apache.hadoop.yarn.util.RackResolver import org.apache.log4j.{Level, Logger} -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{SparkEnv, Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.util.AkkaUtils /** * YarnAllocator is charged with requesting containers from the YARN ResourceManager and deciding @@ -106,7 +107,9 @@ private[yarn] class YarnAllocator( new ThreadFactoryBuilder().setNameFormat("ContainerLauncher #%d").setDaemon(true).build()) launcherPool.allowCoreThreadTimeOut(true) - private val driverUrl = "akka.tcp://sparkDriver@%s:%s/user/%s".format( + private val driverUrl = AkkaUtils.address( + AkkaUtils.protocol(securityMgr.akkaSSLOptions.enabled), + SparkEnv.driverActorSystemName, sparkConf.get("spark.driver.host"), sparkConf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) @@ -300,6 +303,7 @@ private[yarn] class YarnAllocator( assert(container.getResource.getMemory >= resource.getMemory) logInfo("Launching container %s for on host %s".format(containerId, executorHostname)) + executorIdToContainer(executorId) = container val containerSet = allocatedHostToContainersMap.getOrElseUpdate(executorHostname, new HashSet[ContainerId]) diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index d79b85e867fcd..7165918e1bfcf 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -45,6 +45,29 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit |log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n """.stripMargin + private val TEST_PYFILE = """ + |import sys + |from operator import add + | + |from pyspark import SparkConf , SparkContext + |if __name__ == "__main__": + | if len(sys.argv) != 3: + | print >> sys.stderr, "Usage: test.py [master] [result file]" + | exit(-1) + | conf = SparkConf() + | conf.setMaster(sys.argv[1]).setAppName("python test in yarn cluster mode") + | sc = SparkContext(conf=conf) + | status = open(sys.argv[2],'w') + | result = "failure" + | rdd = sc.parallelize(range(10)) + | cnt = rdd.count() + | if cnt == 10: + | result = "success" + | status.write(result) + | status.close() + | sc.stop() + """.stripMargin + private var yarnCluster: MiniYARNCluster = _ private var tempDir: File = _ private var fakeSparkJar: File = _ @@ -98,6 +121,9 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit } fakeSparkJar = File.createTempFile("sparkJar", null, tempDir) + val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) + sys.props += ("spark.yarn.appMasterEnv.SPARK_HOME" -> sparkHome) + sys.props += ("spark.executorEnv.SPARK_HOME" -> sparkHome) sys.props += ("spark.yarn.jar" -> ("local:" + fakeSparkJar.getAbsolutePath())) sys.props += ("spark.executor.instances" -> "1") sys.props += ("spark.driver.extraClassPath" -> childClasspath) @@ -146,6 +172,24 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit assert(Utils.exceptionString(exception).contains("Application finished with failed status")) } + test("run Python application in yarn-cluster mode") { + val primaryPyFile = new File(tempDir, "test.py") + Files.write(TEST_PYFILE, primaryPyFile, Charsets.UTF_8) + val pyFile = new File(tempDir, "test2.py") + Files.write(TEST_PYFILE, pyFile, Charsets.UTF_8) + var result = File.createTempFile("result", null, tempDir) + + val args = Array("--class", "org.apache.spark.deploy.PythonRunner", + "--primary-py-file", primaryPyFile.getAbsolutePath(), + "--py-files", pyFile.getAbsolutePath(), + "--arg", "yarn-cluster", + "--arg", result.getAbsolutePath(), + "--name", "python test in yarn-cluster mode", + "--num-executors", "1") + Client.main(args) + checkResult(result) + } + /** * This is a workaround for an issue with yarn-cluster mode: the Client class will not provide * any sort of error when the job process finishes successfully, but the job itself fails. So
Property NameDefaultMeaning