From 2c06e3890c24f430338f9e3f95f5c0443d0246e2 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 3 Jun 2014 16:11:54 +0800 Subject: [PATCH 01/21] [WIP]initial for sharkclidriver compabiled cli implementation --- src/main/scala/shark/CatalystContext.scala | 170 +++ src/main/scala/shark/CatalystDriver.scala | 49 + src/main/scala/shark/CatalystEnv.scala | 144 ++ src/main/scala/shark/LogHelper.scala | 77 ++ src/main/scala/shark/SharkCliDriver.scala | 45 +- src/main/scala/shark/SharkConfVars.scala | 186 +++ .../tachyon/TachyonOffHeapTableWriter.scala | 58 + .../shark/tachyon/TachyonStorageClient.scala | 186 +++ .../scala/shark/tachyon/TachyonTableRDD.scala | 104 ++ src/main/test/0.20S-exclude.txt | 9 + src/main/test/0.20S-include.txt | 48 + src/main/test/README.md | 28 + src/main/test/java/shark/JavaAPISuite.java | 120 ++ src/main/test/java/shark/SharkQTestUtil.java | 188 +++ .../test/java/shark/TestSharkCliDriver.java | 146 +++ src/main/test/scala/shark/CliSuite.scala | 62 + .../scala/shark/ColumnStatsSQLSuite.scala | 129 ++ .../shark/CountDistinctRewriteSuite.scala | 283 ++++ .../test/scala/shark/ReflectionSuite.scala | 112 ++ src/main/test/scala/shark/SQLSuite.scala | 1154 +++++++++++++++++ src/main/test/scala/shark/SharkRunner.scala | 127 ++ .../test/scala/shark/SharkServerSuite.scala | 108 ++ src/main/test/scala/shark/SortSuite.scala | 53 + .../test/scala/shark/TachyonSQLSuite.scala | 436 +++++++ src/main/test/scala/shark/TestUtils.scala | 195 +++ src/main/test/scala/shark/UtilsSuite.scala | 55 + .../execution/HiveStructSerializerSuite.scala | 65 + .../serialization/SerializationSuite.scala | 109 ++ .../ShuffleSerializerSuite.scala | 112 ++ .../shark/memstore2/CachePolicySuite.scala | 131 ++ .../shark/memstore2/ColumnIteratorSuite.scala | 413 ++++++ .../shark/memstore2/ColumnStatsSuite.scala | 232 ++++ .../shark/memstore2/TablePartitionSuite.scala | 121 ++ .../memstore2/column/ColumnTypeSuite.scala | 121 ++ .../CompressedColumnIteratorSuite.scala | 243 ++++ .../column/CompressionAlgorithmSuite.scala | 366 ++++++ .../column/NullableColumnBuilderSuite.scala | 145 +++ .../column/NullableColumnIteratorSuite.scala | 118 ++ .../scala/shark/util/BloomFilterSuite.scala | 35 + src/main/test/tests_fail.txt | 232 ++++ src/main/test/tests_pass.txt | 925 +++++++++++++ 41 files changed, 7616 insertions(+), 24 deletions(-) create mode 100644 src/main/scala/shark/CatalystContext.scala create mode 100644 src/main/scala/shark/CatalystDriver.scala create mode 100755 src/main/scala/shark/CatalystEnv.scala create mode 100755 src/main/scala/shark/LogHelper.scala create mode 100755 src/main/scala/shark/SharkConfVars.scala create mode 100755 src/main/tachyon_enabled/scala/shark/tachyon/TachyonOffHeapTableWriter.scala create mode 100755 src/main/tachyon_enabled/scala/shark/tachyon/TachyonStorageClient.scala create mode 100755 src/main/tachyon_enabled/scala/shark/tachyon/TachyonTableRDD.scala create mode 100755 src/main/test/0.20S-exclude.txt create mode 100755 src/main/test/0.20S-include.txt create mode 100755 src/main/test/README.md create mode 100755 src/main/test/java/shark/JavaAPISuite.java create mode 100755 src/main/test/java/shark/SharkQTestUtil.java create mode 100755 src/main/test/java/shark/TestSharkCliDriver.java create mode 100755 src/main/test/scala/shark/CliSuite.scala create mode 100755 src/main/test/scala/shark/ColumnStatsSQLSuite.scala create mode 100755 src/main/test/scala/shark/CountDistinctRewriteSuite.scala create mode 100755 src/main/test/scala/shark/ReflectionSuite.scala create mode 100755 src/main/test/scala/shark/SQLSuite.scala create mode 100755 src/main/test/scala/shark/SharkRunner.scala create mode 100755 src/main/test/scala/shark/SharkServerSuite.scala create mode 100755 src/main/test/scala/shark/SortSuite.scala create mode 100755 src/main/test/scala/shark/TachyonSQLSuite.scala create mode 100755 src/main/test/scala/shark/TestUtils.scala create mode 100755 src/main/test/scala/shark/UtilsSuite.scala create mode 100755 src/main/test/scala/shark/execution/HiveStructSerializerSuite.scala create mode 100755 src/main/test/scala/shark/execution/serialization/SerializationSuite.scala create mode 100755 src/main/test/scala/shark/execution/serialization/ShuffleSerializerSuite.scala create mode 100755 src/main/test/scala/shark/memstore2/CachePolicySuite.scala create mode 100755 src/main/test/scala/shark/memstore2/ColumnIteratorSuite.scala create mode 100755 src/main/test/scala/shark/memstore2/ColumnStatsSuite.scala create mode 100755 src/main/test/scala/shark/memstore2/TablePartitionSuite.scala create mode 100755 src/main/test/scala/shark/memstore2/column/ColumnTypeSuite.scala create mode 100755 src/main/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala create mode 100755 src/main/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala create mode 100755 src/main/test/scala/shark/memstore2/column/NullableColumnBuilderSuite.scala create mode 100755 src/main/test/scala/shark/memstore2/column/NullableColumnIteratorSuite.scala create mode 100755 src/main/test/scala/shark/util/BloomFilterSuite.scala create mode 100755 src/main/test/tests_fail.txt create mode 100755 src/main/test/tests_pass.txt diff --git a/src/main/scala/shark/CatalystContext.scala b/src/main/scala/shark/CatalystContext.scala new file mode 100644 index 00000000..6f455d8e --- /dev/null +++ b/src/main/scala/shark/CatalystContext.scala @@ -0,0 +1,170 @@ +/* + * 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 +package hive + +import java.util.{ArrayList => JArrayList} +import scala.collection.JavaConversions._ + +import org.apache.hive.service.cli.TableSchema +import org.apache.hadoop.hive.metastore.api.FieldSchema +import org.apache.hadoop.hive.cli.CliSessionState +import org.apache.hadoop.hive.cli.CliDriver +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hadoop.hive.ql.processors.CommandProcessor +import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse +import org.apache.hadoop.hive.ql.Driver + +import org.apache.spark.SparkContext +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.plans.logical.NativeCommand +import org.apache.spark.sql.catalyst.plans.logical.ExplainCommand +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.QueryExecutionException + +import shark.LogHelper + +case class CatalystContext(sc: SparkContext) extends HiveContext(sc) with LogHelper { + private var result: SchemaRDD = _ + + class HiveQLQueryExecution(hql: String) extends QueryExecution { + override def logical: LogicalPlan = HiveQl.parseSql(hql) + override def toString = hql + "\n" + super.toString + + def result(): (Int, Seq[String]) = analyzed match { + case NativeCommand(cmd) => runOnHive(cmd) + case ExplainCommand(plan) => (0, new QueryExecution { val logical = plan }.toString.split("\n")) + case query => + try{ + val result: Seq[Seq[Any]] = toRdd.collect().toSeq + // We need the types so we can output struct field names + val types = analyzed.output.map(_.dataType) + // Reformat to match hive tab delimited output. + (0, result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t")).toSeq) + } catch { + case e: Throwable => { + logError("Error:\n $cmd\n", e) + (-1, Seq[String]()) + } + } + } + } + + def getResultSetSchema: TableSchema = { + logger.warn(s"Result Schema: ${result.queryExecution.analyzed.output}") + if (result.queryExecution.analyzed.output.size == 0) { + new TableSchema(new FieldSchema("Result", "string", "") :: Nil) + } else { + val schema = result.queryExecution.analyzed.output.map { attr => + new FieldSchema(attr.name, org.apache.spark.sql.hive.HiveMetastoreTypes.toMetastoreType(attr.dataType), "") + } + new TableSchema(schema) + } + } + + def runOnHive(cmd: String, maxRows: Int = 1000): (Int, Seq[String]) = { + try { + val cmd_trimmed: String = cmd.trim() + val tokens: Array[String] = cmd_trimmed.split("\\s+") + val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() + val proc: CommandProcessor = CommandProcessorFactory.get(tokens(0), hiveconf) + + SessionState.start(sessionState) + + proc match { + case driver: Driver => + driver.init() + + val results = new JArrayList[String] + val response: CommandProcessorResponse = driver.run(cmd) + // Throw an exception if there is an error in query processing. + if (response.getResponseCode != 0) { + driver.destroy() + throw new QueryExecutionException(response.getErrorMessage) + } + driver.setMaxRows(maxRows) + driver.getResults(results) + driver.destroy() + (0, results) + case _ => + sessionState.out.println(tokens(0) + " " + cmd_1) + (proc.run(cmd_1).getResponseCode, Seq[String]()) + } + } catch { + case e: Exception => + logger.error( + s""" + |====================== + |HIVE FAILURE OUTPUT + |====================== + |${outputBuffer.toString} + |====================== + |END HIVE FAILURE OUTPUT + |====================== + """.stripMargin) + throw e + } + } + + override lazy val hiveconf = new HiveConf(classOf[SessionState]) + override lazy val sessionState = new SessionState(hiveconf) +} + +abstract class Launcher(cc: CatalystContext) { + def execute(hql: String): (Int, Seq[String]) = new cc.HiveQLQueryExecution(hql).result() +} + +private[hive] case class HiveLauncher(cc: CatalystContext) extends Launcher(cc) { + override def execute(hql: String): (Int, Seq[String]) = cc.runOnHive(hql) +} + +private[hive] case class SparkLauncher(cc: CatalystContext) extends Launcher(cc) + +object CatalystContextWrapper { + val EXEC_MODE = "catalyst.exec.mode" + val EXEC_MODE_SPARK = "spark" + val EXEC_MODE_HIVE = "hive" +} + +class CatalystContextWrapper(cc: CatalystContext) { + val candidates = (CatalystContextWrapper.EXEC_MODE_SPARK, SparkLauncher(cc)) :: + (CatalystContextWrapper.EXEC_MODE_HIVE, HiveLauncher(cc)) :: Nil + +// // Use reflection to get access to the two fields. +// val getFormattedDbMethod = classOf[CliDriver].getDeclaredMethod( +// "getFormattedDb", classOf[HiveConf], classOf[CliSessionState]) +// getFormattedDbMethod.setAccessible(true) +// +// val spacesForStringMethod = classOf[CliDriver].getDeclaredMethod( +// "spacesForString", classOf[String]) +// spacesForStringMethod.setAccessible(true) + + def env: (String, org.apache.spark.sql.hive.Launcher) = { + val conf: HiveConf = cc.sessionState.getConf() +// val db = getFormattedDbMethod.invoke(null, conf, ss).asInstanceOf[String] + val cli = conf.get(CatalystContextWrapper.EXEC_MODE, CatalystContextWrapper.EXEC_MODE_SPARK) + + var launcher = candidates.find(_._1 == cli).getOrElse(candidates.head) + +// val promptStr = if (db != null) launcher._1 + db else "" + + (launcher._1, launcher._2) + } +} \ No newline at end of file diff --git a/src/main/scala/shark/CatalystDriver.scala b/src/main/scala/shark/CatalystDriver.scala new file mode 100644 index 00000000..25faa431 --- /dev/null +++ b/src/main/scala/shark/CatalystDriver.scala @@ -0,0 +1,49 @@ +package shark + +import java.util.ArrayList +import org.apache.hadoop.hive.ql.Driver +import org.apache.hadoop.hive.metastore.api.Schema +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse +import org.apache.hadoop.hive.conf.HiveConf + +import org.apache.spark.sql.hive.CatalystContext + +import scala.collection.JavaConversions._ + +class CatalystDriver(hconf: HiveConf) extends Driver { + private val context: CatalystContext = CatalystEnv.cc + private var schema: Schema = _ + private var result: (Int, Seq[String]) = _ + + override def init(): Unit = { + } + + override def run(command: String): CommandProcessorResponse = { + this.result = new context.HiveQLQueryExecution(command).result + + new CommandProcessorResponse(this.result._1) + } + + override def close(): Int = { + result = null + schema = null + + 0 + } + + override def getSchema(): Schema = schema + + override def getResults(res: ArrayList[String]): Boolean = { + if(result == null) { + false + } else { + res.addAll(result._2) + true + } + } + + override def destroy() { + result = null + schema = null + } +} \ No newline at end of file diff --git a/src/main/scala/shark/CatalystEnv.scala b/src/main/scala/shark/CatalystEnv.scala new file mode 100755 index 00000000..ac17f760 --- /dev/null +++ b/src/main/scala/shark/CatalystEnv.scala @@ -0,0 +1,144 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark + +import scala.collection.mutable.{HashMap, HashSet} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.shims.ShimLoader +import org.apache.spark.SparkConf +import org.apache.spark.scheduler.StatsReportListener +import org.apache.spark.SparkContext +import org.apache.spark.sql.hive.CatalystContext + +/** A singleton object for the master program. The slaves should not access this. */ +object CatalystEnv extends LogHelper { + + def init(): CatalystContext = { + if (sc == null) { + initWithCatalystContext() + } + + cc + } + + def fixUncompatibleConf(conf: Configuration) { + if (sc == null) { + init() + } + + val hiveIslocal = ShimLoader.getHadoopShims.isLocalMode(conf) + if (!sc.isLocal && hiveIslocal) { + val warnMessage = "Hive Hadoop shims detected local mode, but Shark is not running locally." + logWarning(warnMessage) + + // Try to fix this without bothering user + val newValue = "Spark_%s".format(System.currentTimeMillis()) + for (k <- Seq("mapred.job.tracker", "mapreduce.framework.name")) { + val v = conf.get(k) + if (v == null || v == "" || v == "local") { + conf.set(k, newValue) + logWarning("Setting %s to '%s' (was '%s')".format(k, newValue, v)) + } + } + + // If still not fixed, bail out + if (ShimLoader.getHadoopShims.isLocalMode(conf)) { + throw new Exception(warnMessage) + } + } + } + + def initWithCatalystContext( + jobName: String = "Shark::" + java.net.InetAddress.getLocalHost.getHostName, + master: String = System.getenv("MASTER")) + : CatalystContext = { + sc = initSparkContext(jobName, master) + + sc.addSparkListener(new StatsReportListener()) + + cc = CatalystContext(sc) + + cc + } + + private def initSparkContext(conf: SparkConf): SparkContext = { + if (sc != null) { + sc.stop() + } + + sc = new SparkContext(conf) + sc.addSparkListener(new StatsReportListener()) + + sc + } + + private def initSparkContext( + jobName: String = "Shark::" + java.net.InetAddress.getLocalHost.getHostName, + master: String = System.getenv("MASTER")): SparkContext = { + if (sc != null) { + sc.stop() + } + + sc = new SparkContext( + if (master == null) "local" else master, + jobName, + System.getenv("SPARK_HOME"), + Nil, + executorEnvVars) + + sc + } + + logDebug("Initializing SharkEnv") + + val executorEnvVars = new HashMap[String, String] + executorEnvVars.put("SPARK_MEM", getEnv("SPARK_MEM")) + executorEnvVars.put("SPARK_CLASSPATH", getEnv("SPARK_CLASSPATH")) + executorEnvVars.put("HADOOP_HOME", getEnv("HADOOP_HOME")) + executorEnvVars.put("JAVA_HOME", getEnv("JAVA_HOME")) + executorEnvVars.put("MESOS_NATIVE_LIBRARY", getEnv("MESOS_NATIVE_LIBRARY")) + executorEnvVars.put("TACHYON_MASTER", getEnv("TACHYON_MASTER")) + executorEnvVars.put("TACHYON_WAREHOUSE_PATH", getEnv("TACHYON_WAREHOUSE_PATH")) + + val activeSessions = new HashSet[String] + + var cc: CatalystContext = _ + var sc: SparkContext = _ + + // The following line turns Kryo serialization debug log on. It is extremely chatty. + //com.esotericsoftware.minlog.Log.set(com.esotericsoftware.minlog.Log.LEVEL_DEBUG) + + // Keeps track of added JARs and files so that we don't add them twice in consecutive queries. + val addedFiles = HashSet[String]() + val addedJars = HashSet[String]() + + /** Cleans up and shuts down the Shark environments. */ + def stop() { + logDebug("Shutting down Shark Environment") + // Stop the SparkContext + if (CatalystEnv.sc != null) { + sc.stop() + sc = null + cc = null + } + } + + /** Return the value of an environmental variable as a string. */ + def getEnv(varname: String) = if (System.getenv(varname) == null) "" else System.getenv(varname) + +} diff --git a/src/main/scala/shark/LogHelper.scala b/src/main/scala/shark/LogHelper.scala new file mode 100755 index 00000000..e0051467 --- /dev/null +++ b/src/main/scala/shark/LogHelper.scala @@ -0,0 +1,77 @@ +/* + * 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 shark + +import java.io.PrintStream + +import org.apache.commons.lang.StringUtils +import org.apache.hadoop.hive.ql.session.SessionState + +/** + * Utility trait for classes that want to log data. This wraps around Spark's + * Logging trait. It creates a SLF4J logger for the class and allows logging + * messages at different levels using methods that only evaluate parameters + * lazily if the log level is enabled. + * + * It differs from the Spark's Logging trait in that it can print out the + * error to the specified console of the Hive session. + */ +trait LogHelper extends Logging { + + def logError(msg: => String) = { + errStream().println(msg) + logger.error(msg) + } + + def logWarning(msg: => String) = { + errStream().println(msg) + logger.warn(msg) + } + + def logInfo(msg: => String) = { + errStream().println(msg) + logger.info(msg) + } + + def logDebug(msg: => String) = { + errStream().println(msg) + logger.debug(msg) + } + + def logError(msg: String, detail: String) = { + errStream().println(msg) + logger.error(msg + StringUtils.defaultString(detail)) + } + + def logError(msg: String, exception: Throwable) = { + val err = errStream() + err.println(msg) + exception.printStackTrace(err) + logger.error(msg, exception) + } + + def outStream(): PrintStream = { + val ss = SessionState.get() + if (ss != null && ss.out != null) ss.out else System.out + } + + def errStream(): PrintStream = { + val ss = SessionState.get(); + if (ss != null && ss.err != null) ss.err else System.err + } +} diff --git a/src/main/scala/shark/SharkCliDriver.scala b/src/main/scala/shark/SharkCliDriver.scala index 093ece88..87f55a96 100755 --- a/src/main/scala/shark/SharkCliDriver.scala +++ b/src/main/scala/shark/SharkCliDriver.scala @@ -45,13 +45,12 @@ import org.apache.hadoop.hive.shims.ShimLoader import org.apache.hadoop.io.IOUtils import org.apache.thrift.transport.TSocket -/** FIXME object SharkCliDriver { val SKIP_RDD_RELOAD_FLAG = "-skipRddReload" - private var prompt = "shark" + private var prompt = "catalyst" private var prompt2 = " " // when ';' is not yet seen. - private var transport:TSocket = _ +// private var transport:TSocket = _ installSignalHandler() @@ -64,13 +63,13 @@ object SharkCliDriver { HiveInterruptUtils.add(new HiveInterruptCallback { override def interrupt() { // Handle remote execution mode - if (SharkEnv.sc != null) { - SharkEnv.sc.cancelAllJobs() + if (CatalystEnv.sc != null) { + CatalystEnv.sc.cancelAllJobs() } else { - if (transport != null) { - // Force closing of TCP connection upon session termination - transport.getSocket().close() - } +// if (transport != null) { +// // Force closing of TCP connection upon session termination +// transport.getSocket().close() +// } } } }) @@ -129,7 +128,7 @@ object SharkCliDriver { Runtime.getRuntime().addShutdownHook( new Thread() { override def run() { - SharkEnv.stop() + CatalystEnv.stop() } } ) @@ -160,7 +159,7 @@ object SharkCliDriver { val cli = new SharkCliDriver(reloadRdds) cli.setHiveVariables(oproc.getHiveVariables()) - SharkEnv.fixUncompatibleConf(conf) + CatalystEnv.fixUncompatibleConf(conf) // Execute -i init files (always in silent mode) cli.processInitFiles(ss) @@ -214,7 +213,7 @@ object SharkCliDriver { val clientTransportTSocketField = classOf[CliSessionState].getDeclaredField("transport") clientTransportTSocketField.setAccessible(true) - transport = clientTransportTSocketField.get(ss).asInstanceOf[TSocket] +// transport = clientTransportTSocketField.get(ss).asInstanceOf[TSocket] var ret = 0 @@ -265,19 +264,18 @@ class SharkCliDriver(reloadRdds: Boolean = true) extends CliDriver with LogHelpe // Force initializing SharkEnv. This is put here but not object SharkCliDriver // because the Hive unit tests do not go through the main() code path. if (!ss.isRemoteMode()) { - SharkEnv.init() - if (reloadRdds) { - console.printInfo( - "Reloading cached RDDs from previous Shark sessions... (use %s flag to skip reloading)" - .format(SharkCliDriver.SKIP_RDD_RELOAD_FLAG)) - TableRecovery.reloadRdds(processCmd(_), Some(console)) - } + CatalystEnv.init() +// if (reloadRdds) { +// console.printInfo( +// "Reloading cached RDDs from previous Shark sessions... (use %s flag to skip reloading)" +// .format(SharkCliDriver.SKIP_RDD_RELOAD_FLAG)) +// TableRecovery.reloadRdds(processCmd(_), Some(console)) +// } } def this() = this(false) override def processCmd(cmd: String): Int = { - val ss: SessionState = SessionState.get() val cmd_trimmed: String = cmd.trim() val tokens: Array[String] = cmd_trimmed.split("\\s+") val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() @@ -310,8 +308,8 @@ class SharkCliDriver(reloadRdds: Boolean = true) extends CliDriver with LogHelpe // SharkDriver for every command. But it saves us the hassle of // hacking CommandProcessorFactory. val qp: Driver = - if (SharkConfVars.getVar(conf, SharkConfVars.EXEC_MODE) == "shark") { - new SharkDriver(hconf) + if (SharkConfVars.getVar(conf, SharkConfVars.EXEC_MODE) == "catalyst") { + new CatalystDriver(hconf) } else { proc.asInstanceOf[Driver] } @@ -365,7 +363,7 @@ class SharkCliDriver(reloadRdds: Boolean = true) extends CliDriver with LogHelpe } // Destroy the driver to release all the locks. - if (qp.isInstanceOf[SharkDriver]) { + if (qp.isInstanceOf[CatalystDriver]) { qp.destroy() } @@ -403,4 +401,3 @@ class SharkCliDriver(reloadRdds: Boolean = true) extends CliDriver with LogHelpe } } -*/ \ No newline at end of file diff --git a/src/main/scala/shark/SharkConfVars.scala b/src/main/scala/shark/SharkConfVars.scala new file mode 100755 index 00000000..1c04a732 --- /dev/null +++ b/src/main/scala/shark/SharkConfVars.scala @@ -0,0 +1,186 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark + +import scala.language.existentials + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.conf.HiveConf + + +object SharkConfVars { + + val EXEC_MODE = new ConfVar("shark.exec.mode", "catalyst") + + // This is created for testing. Hive's test script assumes a certain output + // format. To pass the test scripts, we need to use Hive's EXPLAIN. + val EXPLAIN_MODE = new ConfVar("shark.explain.mode", "catalyst") + + // If true, keys that are NULL are equal. For strict SQL standard, set this to true. + val JOIN_CHECK_NULL = new ConfVar("shark.join.checknull", true) + + val COLUMNAR_COMPRESSION = new ConfVar("shark.column.compress", true) + + // If true, then cache any table whose name ends in "_cached". + val CHECK_TABLENAME_FLAG = new ConfVar("shark.cache.flag.checkTableName", true) + + // Specify the initial capacity for ArrayLists used to represent columns in columnar + // cache. The default -1 for non-local mode means that Shark will try to estimate + // the number of rows by using: partition_size / (num_columns * avg_field_size). + val COLUMN_BUILDER_PARTITION_SIZE = new ConfVar("shark.column.partitionSize.mb", + if (System.getenv("MASTER") == null) 1 else -1) + + // Prune map splits for cached tables based on predicates in queries. + val MAP_PRUNING = new ConfVar("shark.mappruning", true) + + // Print debug information for map pruning. + val MAP_PRUNING_PRINT_DEBUG = new ConfVar("shark.mappruning.debug", false) + + // If true, then query plans are compressed before being sent + val COMPRESS_QUERY_PLAN = new ConfVar("shark.queryPlan.compress", true) + + // Number of mappers to force for table scan jobs + val NUM_MAPPERS = new ConfVar("shark.map.tasks", -1) + + // Add Shark configuration variables and their default values to the given conf, + // so default values show up in 'set'. + def initializeWithDefaults(conf: Configuration) { + if (conf.get(EXEC_MODE.varname) == null) { + conf.set(EXEC_MODE.varname, EXEC_MODE.defaultVal) + } + if (conf.get(EXPLAIN_MODE.varname) == null) { + conf.set(EXPLAIN_MODE.varname, EXPLAIN_MODE.defaultVal) + } + if (conf.get(COLUMN_BUILDER_PARTITION_SIZE.varname) == null) { + conf.setInt(COLUMN_BUILDER_PARTITION_SIZE.varname, + COLUMN_BUILDER_PARTITION_SIZE.defaultIntVal) + } + if (conf.get(COLUMNAR_COMPRESSION.varname) == null) { + conf.setBoolean(COLUMNAR_COMPRESSION.varname, COLUMNAR_COMPRESSION.defaultBoolVal) + } + if (conf.get(CHECK_TABLENAME_FLAG.varname) == null) { + conf.setBoolean(CHECK_TABLENAME_FLAG.varname, CHECK_TABLENAME_FLAG.defaultBoolVal) + } + if (conf.get(COMPRESS_QUERY_PLAN.varname) == null) { + conf.setBoolean(COMPRESS_QUERY_PLAN.varname, COMPRESS_QUERY_PLAN.defaultBoolVal) + } + if (conf.get(MAP_PRUNING.varname) == null) { + conf.setBoolean(MAP_PRUNING.varname, MAP_PRUNING.defaultBoolVal) + } + if (conf.get(MAP_PRUNING_PRINT_DEBUG.varname) == null) { + conf.setBoolean(MAP_PRUNING_PRINT_DEBUG.varname, MAP_PRUNING_PRINT_DEBUG.defaultBoolVal) + } + } + + def getIntVar(conf: Configuration, variable: ConfVar): Int = { + require(variable.valClass == classOf[Int]) + conf.getInt(variable.varname, variable.defaultIntVal) + } + + def getLongVar(conf: Configuration, variable: ConfVar): Long = { + require(variable.valClass == classOf[Long]) + conf.getLong(variable.varname, variable.defaultLongVal) + } + + def getFloatVar(conf: Configuration, variable: ConfVar): Float = { + require(variable.valClass == classOf[Float]) + conf.getFloat(variable.varname, variable.defaultFloatVal) + } + + def getBoolVar(conf: Configuration, variable: ConfVar): Boolean = { + require(variable.valClass == classOf[Boolean]) + conf.getBoolean(variable.varname, variable.defaultBoolVal) + } + + def getVar(conf: Configuration, variable: ConfVar): String = { + require(variable.valClass == classOf[String]) + conf.get(variable.varname, variable.defaultVal) + } + + def setVar(conf: Configuration, variable: ConfVar, value: String) { + require(variable.valClass == classOf[String]) + conf.set(variable.varname, value) + } + + def getIntVar(conf: Configuration, variable: HiveConf.ConfVars): Int = { + HiveConf.getIntVar(conf, variable) + } + + def getLongVar(conf: Configuration, variable: HiveConf.ConfVars): Long = { + HiveConf.getLongVar(conf, variable) + } + + def getLongVar(conf: Configuration, variable: HiveConf.ConfVars, defaultVal: Long): Long = { + HiveConf.getLongVar(conf, variable, defaultVal) + } + + def getFloatVar(conf: Configuration, variable: HiveConf.ConfVars): Float = { + HiveConf.getFloatVar(conf, variable) + } + + def getFloatVar(conf: Configuration, variable: HiveConf.ConfVars, defaultVal: Float): Float = { + HiveConf.getFloatVar(conf, variable, defaultVal) + } + + def getBoolVar(conf: Configuration, variable: HiveConf.ConfVars): Boolean = { + HiveConf.getBoolVar(conf, variable) + } + + def getBoolVar(conf: Configuration, variable: HiveConf.ConfVars, defaultVal: Boolean): Boolean = { + HiveConf.getBoolVar(conf, variable, defaultVal) + } + + def getVar(conf: Configuration, variable: HiveConf.ConfVars): String = { + HiveConf.getVar(conf, variable) + } + + def getVar(conf: Configuration, variable: HiveConf.ConfVars, defaultVal: String): String = { + HiveConf.getVar(conf, variable, defaultVal) + } +} + + +case class ConfVar( + varname: String, + valClass: Class[_], + defaultVal: String, + defaultIntVal: Int, + defaultLongVal: Long, + defaultFloatVal: Float, + defaultBoolVal: Boolean) { + + def this(varname: String, defaultVal: String) = { + this(varname, classOf[String], defaultVal, 0, 0, 0, false) + } + + def this(varname: String, defaultVal: Int) = { + this(varname, classOf[Int], defaultVal.toString, defaultVal, 0, 0, false) + } + + def this(varname: String, defaultVal: Long) = { + this(varname, classOf[Long], defaultVal.toString, 0, defaultVal, 0, false) + } + + def this(varname: String, defaultVal: Float) = { + this(varname, classOf[Float], defaultVal.toString, 0, 0, defaultVal, false) + } + + def this(varname: String, defaultVal: Boolean) = { + this(varname, classOf[Boolean], defaultVal.toString, 0, 0, 0, defaultVal) + } +} diff --git a/src/main/tachyon_enabled/scala/shark/tachyon/TachyonOffHeapTableWriter.scala b/src/main/tachyon_enabled/scala/shark/tachyon/TachyonOffHeapTableWriter.scala new file mode 100755 index 00000000..522f377c --- /dev/null +++ b/src/main/tachyon_enabled/scala/shark/tachyon/TachyonOffHeapTableWriter.scala @@ -0,0 +1,58 @@ +/* +* Copyright (C) 2012 The Regents of The University California. +* All rights reserved. +* +* Licensed 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 shark.tachyon + +import java.nio.ByteBuffer + +import tachyon.client.WriteType + +import shark.LogHelper +import shark.execution.serialization.JavaSerializer +import shark.memstore2.{OffHeapStorageClient, OffHeapTableWriter, TablePartitionStats} + +class TachyonOffHeapTableWriter(@transient path: String, @transient numColumns: Int) + extends OffHeapTableWriter with LogHelper { + + // Re-instantiated upon deserialization, the first time it's referenced. + @transient lazy val tfs = OffHeapStorageClient.client.asInstanceOf[TachyonStorageClient].tfs + + var rawTableId: Int = -1 + + override def createTable() { + val metadata = ByteBuffer.allocate(0) + rawTableId = tfs.createRawTable(path, numColumns, metadata) + } + + override def setStats(indexToStats: collection.Map[Int, TablePartitionStats]) { + val buffer = ByteBuffer.wrap(JavaSerializer.serialize(indexToStats)) + tfs.updateRawTableMetadata(rawTableId, buffer) + } + + // rawTable is a lazy val so it gets created the first time it is referenced. + // This is only used on worker nodes. + @transient lazy val rawTable = tfs.getRawTable(rawTableId) + + override def writeColumnPartition(column: Int, part: Int, data: ByteBuffer) { + val rawColumn = rawTable.getRawColumn(column) + rawColumn.createPartition(part) + val file = rawColumn.getPartition(part) + val outStream = file.getOutStream(WriteType.CACHE_THROUGH) + outStream.write(data.array(), 0, data.limit()) + outStream.close() + } +} diff --git a/src/main/tachyon_enabled/scala/shark/tachyon/TachyonStorageClient.scala b/src/main/tachyon_enabled/scala/shark/tachyon/TachyonStorageClient.scala new file mode 100755 index 00000000..a494578d --- /dev/null +++ b/src/main/tachyon_enabled/scala/shark/tachyon/TachyonStorageClient.scala @@ -0,0 +1,186 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.tachyon + +import java.util.{BitSet => JBitSet} +import java.util.concurrent.{ConcurrentHashMap => ConcurrentJavaHashMap} + +import scala.collection.JavaConversions._ + +import org.apache.spark.rdd.{RDD, UnionRDD} +import tachyon.client.TachyonFS + +import shark.{LogHelper, SharkEnv} +import shark.execution.TableReader.PruningFunctionType +import shark.execution.serialization.JavaSerializer +import shark.memstore2.{OffHeapStorageClient, OffHeapStorageClientFactory, TablePartitionStats} + +class TachyonStorageClientFactory extends OffHeapStorageClientFactory { + def createClient() = { + new TachyonStorageClient( + System.getenv("TACHYON_MASTER"), System.getenv("TACHYON_WAREHOUSE_PATH")) + } +} + +/** + * Enables caching of Shark tables in native column-oriented format into Tachyon. + * + * The directory structure for a given table in Tachyon looks like: + * Data: warehouse/database.table/_defaultkey/insert_#/col_#/part_# + * Metadata: warehouse/database.table/_defaultkey/insert_#/.meta + * where: + * - insert_# is used to allow inserting data multiple times. Files cannot be appended to in + * Tachyon, so we instead create a whole new directory and union them upon read. + * - col_# is the folder for the particular column + * - part_# is the Spark partition for the column + * + * Note that "_defaultkey" is the name of the Hive Partition for a non-partitioned table. If the + * table is partitioned, it will be replaced by each "hivePartitionKey". + */ +class TachyonStorageClient(val master: String, val warehousePath: String) + extends OffHeapStorageClient with LogHelper { + + /** We create a new directory with a new RawTable for each independent insert. */ + private val INSERT_FILE_PREFIX = "insert_" + + /** Non-partitioned tables use a default partition name for consistency. */ + private val DEFAULT_PARTITION = "_defaultkey" + + private val _fileNameMappings = new ConcurrentJavaHashMap[String, Int]() + + if (master == null || master == "") { + throw new TachyonException("TACHYON_MASTER is not set, cannot create TachyonStorageClient.") + } + if (warehousePath == null) { + throw new TachyonException("TACHYON_WAREHOUSE is not set, cannot create TachyonStorageClient.") + } + + val tfs = TachyonFS.get(master) + + private def getUniqueFilePath(parentDirectory: String): String = { + val parentDirectoryLower = parentDirectory.toLowerCase + val currentInsertNum = if (_fileNameMappings.containsKey(parentDirectoryLower)) { + _fileNameMappings.get(parentDirectoryLower) + } else { + 0 + } + var nextInsertNum = currentInsertNum + 1 + val filePath = parentDirectoryLower + "/" + INSERT_FILE_PREFIX + // Make sure there aren't file conflicts. This could occur if the directory was created in a + // previous Shark session. + while (tfs.exist(filePath + nextInsertNum)) { + nextInsertNum = nextInsertNum + 1 + } + _fileNameMappings.put(parentDirectoryLower, nextInsertNum) + filePath + nextInsertNum + } + + private def getTablePath(tableKey: String): String = { + warehousePath + "/" + tableKey + } + + private def getPartitionPath(tableKey: String, hivePartitionKey: String): String = { + getTablePath(tableKey) + "/" + hivePartitionKey + } + + override def tableExists(tableKey: String): Boolean = { + tfs.exist(getTablePath(tableKey)) + } + + override def tablePartitionExists(tableKey: String, hivePartitionKey: Option[String]): Boolean = { + tfs.exist(getPartitionPath(tableKey, hivePartitionKey.getOrElse(DEFAULT_PARTITION))) + } + + override def dropTable(tableKey: String): Boolean = { + tfs.delete(getTablePath(tableKey), true /* recursively */) + } + + override def dropTablePartition(tableKey: String, hivePartitionKey: Option[String]): Boolean = { + tfs.delete(getPartitionPath(tableKey, hivePartitionKey.getOrElse(DEFAULT_PARTITION)), + true /* recursively */) + } + + /** + * Reads a particular Hive partition (or whole table if non-partitioned) into a single RDD. + * Since each insert is written to its own directory, we need to reconstruct this RDD by reading + * across all of these insert directories. We thus column- and row-prune our data before + * producing the union to avoid data transfer. + * + * @param columnsUsed Indicates which columns are needed, to avoid loading extra data. + * @param pruningFn Used for pruning rows. + */ + override def readTablePartition( + tableKey: String, + hivePartitionKey: Option[String], + columnsUsed: JBitSet, + pruningFn: PruningFunctionType + ): RDD[_] = { + + try { + if (!tablePartitionExists(tableKey, hivePartitionKey)) { + throw new TachyonException("Table " + tableKey + " does not exist in Tachyon") + } + + // Create a TachyonTableRDD for each raw tableRDDsAndStats file in the directory. + val tableDirectory = getPartitionPath(tableKey, hivePartitionKey.getOrElse(DEFAULT_PARTITION)) + val files = tfs.ls(tableDirectory, false /* recursive */) + // The first path is just "{tableDirectory}/", so ignore it. + val rawTableFiles = files.subList(1, files.size) + val prunedRDDs = rawTableFiles.map { filePath => + val serializedMetadata = tfs.getRawTable(tfs.getFileId(filePath)).getMetadata + val indexToStats = JavaSerializer.deserialize[collection.Map[Int, TablePartitionStats]]( + serializedMetadata.array()) + pruningFn(new TachyonTableRDD(filePath, columnsUsed, SharkEnv.sc), indexToStats) + } + new UnionRDD(SharkEnv.sc, prunedRDDs.toSeq.asInstanceOf[Seq[RDD[Any]]]) + } catch { + case e: Exception => + logError("Exception while reading table partition", e) + throw e + } + } + + override def createTablePartitionWriter( + tableKey: String, + hivePartitionKey: Option[String], + numColumns: Int): TachyonOffHeapTableWriter = { + if (!tfs.exist(warehousePath)) { + tfs.mkdir(warehousePath) + } + val parentDirectory = getPartitionPath(tableKey, hivePartitionKey.getOrElse(DEFAULT_PARTITION)) + val filePath = getUniqueFilePath(parentDirectory) + new TachyonOffHeapTableWriter(filePath, numColumns) + } + + override def createTablePartition( + tableKey: String, + hivePartitionKeyOpt: Option[String]): Boolean = { + hivePartitionKeyOpt match { + case Some(key) => tfs.mkdir(getPartitionPath(tableKey, key)) + case None => tfs.mkdir(getPartitionPath(tableKey, DEFAULT_PARTITION)) + } + } + + override def renameTable( + oldTableKey: String, + newTableKey: String): Boolean = { + val oldPath = getTablePath(oldTableKey) + val newPath = getTablePath(newTableKey) + tfs.rename(oldPath, newPath) + } +} diff --git a/src/main/tachyon_enabled/scala/shark/tachyon/TachyonTableRDD.scala b/src/main/tachyon_enabled/scala/shark/tachyon/TachyonTableRDD.scala new file mode 100755 index 00000000..59ca6b0c --- /dev/null +++ b/src/main/tachyon_enabled/scala/shark/tachyon/TachyonTableRDD.scala @@ -0,0 +1,104 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.tachyon + +import java.nio.{ByteBuffer, ByteOrder} +import java.util.{BitSet => JBitSet} + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.{Partition, SparkContext, TaskContext} +import shark.memstore2._ +import tachyon.client.{ReadType, TachyonByteBuffer} +import tachyon.client.table.RawTable +import org.apache.spark.rdd.RDD +import shark.LogHelper + +private class TachyonTablePartition(rddId: Int, idx: Int, val locations: Seq[String]) + extends Partition { + + override def hashCode(): Int = (41 * (41 + rddId) + idx).toInt + + override val index: Int = idx +} + +/** + * An RDD that reads a Tachyon Table. + */ +class TachyonTableRDD(path: String, columnsUsed: JBitSet, @transient sc: SparkContext) + extends RDD[TablePartition](sc, Nil) with LogHelper { + + @transient lazy val tfs = OffHeapStorageClient.client.asInstanceOf[TachyonStorageClient].tfs + + override def getPartitions: Array[Partition] = { + val rawTable: RawTable = tfs.getRawTable(path) + // Use the first column to get preferred locations for all partitions. + val rawColumn = rawTable.getRawColumn(0) + val numPartitions: Int = rawColumn.partitions() + Array.tabulate[Partition](numPartitions) { part => + val locations = rawColumn.getPartition(part).getLocationHosts().asScala + new TachyonTablePartition(id, part, locations) : Partition + } + } + + override def compute(theSplit: Partition, context: TaskContext): Iterator[TablePartition] = { + val rawTable: RawTable = tfs.getRawTable(path) + val activeBuffers = new ArrayBuffer[TachyonByteBuffer]() + val buffers = Array.tabulate[ByteBuffer](rawTable.getColumns()) { columnIndex => + if (columnIndex != 0 && columnsUsed != null && !columnsUsed.get(columnIndex - 1)) { + null + } else { + val fp = rawTable.getRawColumn(columnIndex).getPartition(theSplit.index, true) + // Try to read data from Tachyon's memory, either local or remote. + var buf = fp.readByteBuffer() + if (buf == null && fp.recache()) { + // The data is not in Tachyon's memory yet, recache succeed. + buf = fp.readByteBuffer() + } + if (buf == null) { + logWarning("Table " + path + " column " + columnIndex + " partition " + theSplit.index + + " is not in Tachyon's memory. Streaming it in.") + var data = ByteBuffer.allocate(fp.length().toInt) + val is = fp.getInStream(ReadType.CACHE) + is.read(data.array) + is.close() + data.limit(fp.length().toInt) + data + } else { + activeBuffers += buf + buf.DATA + } + } + } + + // Register an on-task-completion callback to close the input stream. + context.addOnCompleteCallback(() => activeBuffers.foreach(_.close())) + + Iterator(new TablePartition(buffers.map(buffer => + if (buffer == null) null else buffer.order(ByteOrder.nativeOrder())))) + } + + override def getPreferredLocations(split: Partition): Seq[String] = { + split.asInstanceOf[TachyonTablePartition].locations + } + + // override def checkpoint() { + // // Do nothing. Tachyon RDD should not be checkpointed. + // } +} diff --git a/src/main/test/0.20S-exclude.txt b/src/main/test/0.20S-exclude.txt new file mode 100755 index 00000000..92a75082 --- /dev/null +++ b/src/main/test/0.20S-exclude.txt @@ -0,0 +1,9 @@ +testCliDriver_archive_excludeHadoop20 +testCliDriver_auto_join14 +testCliDriver_combine2 +testCliDriver_ctas +testCliDriver_input12 +testCliDriver_input39 +testCliDriver_join14 +testCliDriver_loadpart_err +testCliDriver_sample_islocalmode_hook \ No newline at end of file diff --git a/src/main/test/0.20S-include.txt b/src/main/test/0.20S-include.txt new file mode 100755 index 00000000..d15b64ef --- /dev/null +++ b/src/main/test/0.20S-include.txt @@ -0,0 +1,48 @@ +testCliDriver_archive +testCliDriver_archive_corrupt +testCliDriver_infer_bucket_sort_list_bucket +testCliDriver_list_bucket_dml_1 +testCliDriver_list_bucket_dml_11 +testCliDriver_list_bucket_dml_12 +testCliDriver_list_bucket_dml_13 +testCliDriver_list_bucket_dml_2 +testCliDriver_list_bucket_dml_3 +testCliDriver_list_bucket_dml_4 +testCliDriver_list_bucket_dml_5 +testCliDriver_list_bucket_dml_6 +testCliDriver_list_bucket_dml_7 +testCliDriver_list_bucket_dml_8 +testCliDriver_list_bucket_dml_9 +testCliDriver_list_bucket_query_multiskew_1 +testCliDriver_list_bucket_query_multiskew_2 +testCliDriver_list_bucket_query_multiskew_3 +testCliDriver_list_bucket_query_oneskew_1 +testCliDriver_list_bucket_query_oneskew_2 +testCliDriver_list_bucket_query_oneskew_3 +testCliDriver_recursive_dir +testCliDriver_skewjoin_union_remove_1 +testCliDriver_skewjoin_union_remove_2 +testCliDriver_split_sample +testCliDriver_union_remove_1 +testCliDriver_union_remove_10 +testCliDriver_union_remove_11 +testCliDriver_union_remove_12 +testCliDriver_union_remove_13 +testCliDriver_union_remove_14 +testCliDriver_union_remove_15 +testCliDriver_union_remove_16 +testCliDriver_union_remove_17 +testCliDriver_union_remove_18 +testCliDriver_union_remove_19 +testCliDriver_union_remove_2 +testCliDriver_union_remove_20 +testCliDriver_union_remove_21 +testCliDriver_union_remove_22 +testCliDriver_union_remove_23 +testCliDriver_union_remove_24 +testCliDriver_union_remove_3 +testCliDriver_union_remove_4 +testCliDriver_union_remove_5 +testCliDriver_union_remove_7 +testCliDriver_union_remove_8 +testCliDriver_union_remove_9 \ No newline at end of file diff --git a/src/main/test/README.md b/src/main/test/README.md new file mode 100755 index 00000000..1ae3b814 --- /dev/null +++ b/src/main/test/README.md @@ -0,0 +1,28 @@ +###Hive Compatibility Test Warnings + +#### Test results that rely on tables with `timestamp` fields may differ across JVM versions. +For example, these tests: +* udf5 +* timestamp.1, timestamp_2, timestamp_udf + +Pass when running with this JVM: +(Mac 10.9, AMPLab Jenkins) +java version "1.7.0_25" +Java(TM) SE Runtime Environment (build 1.7.0_25-b15) +Java HotSpot(TM) 64-Bit Server VM (build 23.25-b01, mixed mode) + +But fail on EC2 when run with this JVM: +(EC2 c2.2xlarge) +java version "1.7.0_45" +OpenJDK Runtime Environment (amzn-2.4.3.2.32.amzn1-x86_64 u45-b15) +OpenJDK 64-Bit Server VM (build 24.45-b08, mixed mode) + + +A few more tests from test_pass.txt that fall into this category: +TestCliDriver_input_part8 +TestSharkCliDriver: testCliDriver_timestamp_1 +TestSharkCliDriver: testCliDriver_timestamp_2 +TestSharkCliDriver: testCliDriver_timestamp_3 +TestSharkCliDriver: testCliDriver_timestamp_udf +TestSharkCliDriver: testCliDriver_udf_to_unix_timestamp +TestSharkCliDriver: testCliDriver_udf5 diff --git a/src/main/test/java/shark/JavaAPISuite.java b/src/main/test/java/shark/JavaAPISuite.java new file mode 100755 index 00000000..8e86890d --- /dev/null +++ b/src/main/test/java/shark/JavaAPISuite.java @@ -0,0 +1,120 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark; + +import java.io.Serializable; +import java.util.List; + +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.Function; + +import shark.api.Row; +import shark.api.JavaSharkContext; +import shark.api.JavaTableRDD; + + +// The test suite itself is Serializable so that anonymous Function implementations can be +// serialized, as an alternative to converting these anonymous classes to static inner classes; +// see http://stackoverflow.com/questions/758570/. +public class JavaAPISuite implements Serializable { + + private static final String WAREHOUSE_PATH = TestUtils$.MODULE$.getWarehousePath(); + private static final String METASTORE_PATH = TestUtils$.MODULE$.getMetastorePath(); + + private static transient JavaSharkContext sc; + + @BeforeClass + public static void oneTimeSetUp() { + // Intentionally leaving this here since SBT doesn't seem to display junit tests well ... + System.out.println("running JavaAPISuite ================================================"); + + // Check if the SharkEnv's SharkContext has already been initialized. If so, use that to + // instantiate a JavaSharkContext. + sc = SharkRunner.initWithJava(); + + // test + sc.sql("drop table if exists test_java"); + sc.sql("CREATE TABLE test_java (key INT, val STRING)"); + sc.sql("LOAD DATA LOCAL INPATH '${hiveconf:shark.test.data.path}/kv1.txt' INTO TABLE test_java"); + + // users + sc.sql("drop table if exists users_java"); + sc.sql("create table users_java (id int, name string) row format delimited fields terminated by '\t'"); + sc.sql("load data local inpath '${hiveconf:shark.test.data.path}/users.txt' OVERWRITE INTO TABLE users_java"); + } + + @AfterClass + public static void oneTimeTearDown() { + sc.stop(); + System.clearProperty("spark.driver.port"); + } + + @Test + public void selectQuery() { + List result = sc.sql("select val from test_java"); + Assert.assertEquals(500, result.size()); + Assert.assertTrue(result.contains("val_407")); + } + + @Test + public void sql2rdd() { + JavaTableRDD result = sc.sql2rdd("select val from test_java"); + JavaRDD values = result.map(new Function() { + @Override + public String call(Row x) { + return x.getString(0); + } + }); + Assert.assertEquals(500, values.count()); + Assert.assertTrue(values.collect().contains("val_407")); + } + + @Test + public void filter() { + JavaTableRDD result = sc.sql2rdd("select * from users_java"); + JavaTableRDD filtered = result.filter(new Function() { + @Override + public Boolean call(Row row) throws Exception { + return row.getString("name").equals("B"); + } + }).cache(); + Assert.assertEquals(1, filtered.count()); + Assert.assertEquals(2, filtered.first().getInt("id").intValue()); + } + + @Test + public void union() { + JavaTableRDD a = sc.sql2rdd("select * from users_java where name = \"A\""); + JavaTableRDD b = sc.sql2rdd("select * from users_java where name = \"B\""); + JavaTableRDD union = a.union(b); + Assert.assertEquals(3, union.count()); + List uniqueNames = union.map(new Function() { + @Override + public String call(Row row) throws Exception { + return row.getString("name"); + } + }).distinct().collect(); + Assert.assertEquals(2, uniqueNames.size()); + } + +} diff --git a/src/main/test/java/shark/SharkQTestUtil.java b/src/main/test/java/shark/SharkQTestUtil.java new file mode 100755 index 00000000..67f69fe1 --- /dev/null +++ b/src/main/test/java/shark/SharkQTestUtil.java @@ -0,0 +1,188 @@ +package shark; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.PrintStream; +import java.lang.reflect.Method; + +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.hive.cli.CliSessionState; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.QTestUtil; +import org.apache.hadoop.hive.ql.exec.Utilities.StreamPrinter; +import org.apache.hadoop.hive.ql.session.SessionState; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +/** + * Replaces Hive's QTestUtil class by using the SharkDriver instead of Hive's + * Driver. Also changes the way comparison is done by forcing a sort and + * truncating floating point numbers. + */ +public class SharkQTestUtil extends QTestUtil { + + private static Method maskPatternsMethod; + + static final private Log LOG = LogFactory.getLog(SharkQTestUtil.class.getName()); + + static { + try { + maskPatternsMethod = QTestUtil.class.getDeclaredMethod("maskPatterns", + String[].class, String.class); + maskPatternsMethod.setAccessible(true); + } catch (SecurityException e) { + e.printStackTrace(); + } catch (NoSuchMethodException e) { + e.printStackTrace(); + } + } + + private SharkCliDriver cliDrv; + + public SharkQTestUtil(String outDir, String logDir) throws Exception { + super(outDir, logDir); + } + + public SharkQTestUtil(String outDir, String logDir, boolean miniMr, + String hadoopVer) throws Exception { + super(outDir, logDir, miniMr, hadoopVer); + } + + @Override + public void cliInit(String tname, boolean recreate) throws Exception { + SharkConfVars.setVar(conf, SharkConfVars.EXPLAIN_MODE(), "hive"); + + if (recreate) { + cleanUp(); + createSources(); + } + + HiveConf.setVar(conf, HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER, + "org.apache.hadoop.hive.ql.security.DummyAuthenticator"); + + // Set the default min number of map tasks to 1 since that's what most Hive tests expects. + conf.setInt("mapred.map.tasks", 1); + + CliSessionState ss = new CliSessionState(conf); + assert(ss != null); + ss.in = System.in; + + File qf = new File(outDir, tname); + File outf = new File(new File(logDir), qf.getName().concat(".out")); + FileOutputStream fo = new FileOutputStream(outf); + ss.out = new PrintStream(fo, true, "UTF-8"); + ss.err = ss.out; + ss.setIsSilent(true); + SessionState oldSs = SessionState.get(); + if (oldSs != null && oldSs.out != null && oldSs.out != System.out) { + oldSs.out.close(); + } + SessionState.start(ss); + + cliDrv = new SharkCliDriver(); + if (tname.equals("init_file.q")) { + File testInitFile = new File("../data/scripts/test_init_file.sql"); + try { + ss.initFiles.add(testInitFile.getAbsolutePath()); + } catch (Exception e) { + System.out.println("Exception is =" + e.getMessage()); + } + } + cliDrv.processInitFiles(ss); + } + + @Override + public int executeClient(String tname) { + LOG.info("Begin query: " + tname); + return cliDrv.processLine(getQMap().get(tname)); + } + + @Override + public int checkCliDriverResults(String tname) throws Exception { + String[] cmdArray; + String[] patterns; + assert(getQMap().containsKey(tname)); + + String outFileName = outPath(outDir, tname + ".out"); + + patterns = new String[] { + ".*file:.*", + ".*pfile:.*", + ".*hdfs:.*", + ".*/tmp/.*", + ".*invalidscheme:.*", + ".*lastUpdateTime.*", + ".*lastAccessTime.*", + ".*lastModifiedTime.*", + ".*[Oo]wner.*", + ".*CreateTime.*", + ".*LastAccessTime.*", + ".*Location.*", + ".*LOCATION '.*", + ".*transient_lastDdlTime.*", + ".*last_modified_.*", + ".*java.lang.RuntimeException.*", + ".*at org.*", + ".*at sun.*", + ".*at java.*", + ".*at junit.*", + ".*Caused by:.*", + ".*LOCK_QUERYID:.*", + ".*LOCK_TIME:.*", + ".*grantTime.*", + ".*[.][.][.] [0-9]* more.*", + ".*job_[0-9]*_[0-9]*.*", + ".*USING 'java -cp.*", + "^Deleted.*", + }; + maskPatternsMethod.invoke(this, patterns, (new File(logDir, tname + ".out")).getPath()); + + cmdArray = new String[] { + "diff", "-a", + "-I", "PREHOOK", + "-I", "POSTHOOK" + }; + + // Only keep 5 digits of precision for floating point numbers. + // Also trim trailing whitespace. + String truncFloatCmd = "perl -p -e 's/(\\d\\.\\d{5})\\d*/\\1/g;' -e 's/\\s+$/\\n/g'"; + String expectedFile = (new File(logDir, tname + ".out")).getPath(); + + String cmdString = "\"" + + StringUtils.join(cmdArray, "\" \"") + "\" " + + "<(sort " + expectedFile + " | " + truncFloatCmd + ") " + + "<(sort " + outFileName + " | " + truncFloatCmd + ")"; + System.out.println("Comparing: " + expectedFile + " " + outFileName); + System.out.println(cmdString); + + //System.out.println(org.apache.commons.lang.StringUtils.join(cmdArray, ' ')); + String[] bashCmdArray = new String[3]; + bashCmdArray[0] = "bash"; + bashCmdArray[1] = "-c"; + bashCmdArray[2] = cmdString; + Process executor = Runtime.getRuntime().exec(bashCmdArray); + + StreamPrinter outPrinter = new StreamPrinter( + executor.getInputStream(), null, SessionState.getConsole().getChildOutStream()); + StreamPrinter errPrinter = new StreamPrinter( + executor.getErrorStream(), null, SessionState.getConsole().getChildErrStream()); + + outPrinter.start(); + errPrinter.start(); + + int exitVal = executor.waitFor(); + + if (exitVal != 0 && overWrite) { + System.out.println("Overwriting results"); + cmdArray = new String[3]; + cmdArray[0] = "cp"; + cmdArray[1] = (new File(logDir, tname + ".out")).getPath(); + cmdArray[2] = outFileName; + executor = Runtime.getRuntime().exec(cmdArray); + exitVal = executor.waitFor(); + } + + return exitVal; + } + +} diff --git a/src/main/test/java/shark/TestSharkCliDriver.java b/src/main/test/java/shark/TestSharkCliDriver.java new file mode 100755 index 00000000..a455092f --- /dev/null +++ b/src/main/test/java/shark/TestSharkCliDriver.java @@ -0,0 +1,146 @@ +package shark; + +import java.io.BufferedReader; +import java.io.FileNotFoundException; +import java.io.FileReader; +import java.io.IOException; +import java.lang.reflect.Field; +import java.util.Enumeration; +import java.util.HashSet; +import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import junit.framework.Test; +import junit.framework.TestCase; +import junit.framework.TestSuite; +import org.junit.Ignore; + +import org.apache.hadoop.hive.cli.TestCliDriver; +import org.apache.hadoop.hive.ql.QTestUtil; + +import java.util.Iterator; + +/** + * The test driver. It overloads Hive's TestCliDriver to use SharkQTestUtil. + * There is also a feature to selectively run tests, i.e. only tests whose + * names match the regular expression pattern defined in environmental variable + * TEST are invoked. Note that this test is ignored so we don't run it in our + * normal unit tests. This should only be launched with the explicit test script. + */ +@Ignore public class TestSharkCliDriver extends TestCliDriver { + + static { + // Replace qt in Hive's TestCliDriver with SharkQTestUtil. + try { + + Field qtField = TestCliDriver.class.getDeclaredField("qt"); + qtField.setAccessible(true); + + Field outDirField = QTestUtil.class.getDeclaredField("outDir"); + outDirField.setAccessible(true); + Field logDirField = QTestUtil.class.getDeclaredField("logDir"); + logDirField.setAccessible(true); + + QTestUtil qt = (QTestUtil) qtField.get(null); + String outDir = (String) outDirField.get(qt); + String logDir = (String) logDirField.get(qt); + + qt = new SharkQTestUtil(outDir, logDir); + // cleanUp / createSources are already called in TestCliDriver. + //qt.cleanUp(); + //qt.createSources(); + + qtField.set(null, qt); + + } catch (Exception e) { + e.printStackTrace(); + } + } + + public TestSharkCliDriver(String name) { + super(name); + } + + public static Test suite() { + TestSuite suite = new TestSuite(); + TestSuite hiveSuite = (TestSuite) TestCliDriver.suite(); + + @SuppressWarnings("unchecked") + Enumeration tests = (Enumeration) hiveSuite.tests(); + + String fileName = System.getenv("TEST_FILE"); + Set regTestsFromFile = new HashSet(); + if (fileName != null && fileName.length() > 0) { + try { + BufferedReader reader = new BufferedReader(new FileReader(fileName)); + String line = null; + while ((line = reader.readLine()) != null) { + regTestsFromFile.add(line); + } + } catch (FileNotFoundException e) { + System.err.println("FileNotFoundException: " + e.getMessage()); + System.exit(1); + } catch (IOException e) { + System.err.println("IOException: " + e.getMessage()); + System.exit(1); + } + } + + Pattern regexPattern = null; + String pattern = System.getenv("TEST"); + if (pattern != null && pattern.length() > 0) { + regexPattern = Pattern.compile(System.getenv("TEST")); + } + + System.out.println("---------------------------------------------------"); + System.out.println("---------------------------------------------------"); + System.out.println("---------------------------------------------------"); + System.out.println("---------------------------------------------------"); + System.out.println(TestSharkCliDriver.class.getName()); + + boolean readFile = (regTestsFromFile.size() != 0); + + while (tests.hasMoreElements()) { + TestCase test = (TestCase) tests.nextElement(); + + boolean passRegex = (regexPattern == null); + boolean passFile = !readFile; + + if (regexPattern != null) { + Matcher m = regexPattern.matcher(test.getName()); + if (m.find() || test.getName() == "testCliDriver_shutdown") { + passRegex = true; + } + } + + if (regTestsFromFile.size() > 0) { + passFile = regTestsFromFile.contains(test.getName()); + } + + if (passRegex && passFile) { + suite.addTest(test); + regTestsFromFile.remove(test.getName()); + System.out.println("TestSharkCliDriver: " + test.getName()); + if (readFile && regTestsFromFile.size() == 0) + break; + } + } + + Iterator regTestsFromFileIter = regTestsFromFile.iterator(); + while (regTestsFromFileIter.hasNext()) { + String test = regTestsFromFileIter.next(); + System.out.println("Warning! Hive test not found: " + test); + } + + System.out.println("TestSharkCliDriver total test to run: " + suite.countTestCases()); + + System.out.println("---------------------------------------------------"); + System.out.println("---------------------------------------------------"); + System.out.println("---------------------------------------------------"); + System.out.println("---------------------------------------------------"); + + + return suite; + } +} diff --git a/src/main/test/scala/shark/CliSuite.scala b/src/main/test/scala/shark/CliSuite.scala new file mode 100755 index 00000000..f2e87796 --- /dev/null +++ b/src/main/test/scala/shark/CliSuite.scala @@ -0,0 +1,62 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark + +import java.io.{BufferedReader, File, InputStreamReader, PrintWriter} +import org.scalatest.{BeforeAndAfterAll, FunSuite} + + +/** + * Test the Shark CLI. + */ +class CliSuite extends FunSuite with BeforeAndAfterAll with TestUtils { + + val WAREHOUSE_PATH = TestUtils.getWarehousePath("cli") + val METASTORE_PATH = TestUtils.getMetastorePath("cli") + + override def beforeAll() { + val pb = new ProcessBuilder( + "./bin/shark", + "-hiveconf", + "javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + METASTORE_PATH + ";create=true", + "-hiveconf", + "hive.metastore.warehouse.dir=" + WAREHOUSE_PATH) + + process = pb.start() + outputWriter = new PrintWriter(process.getOutputStream, true) + inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) + errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) + waitForOutput(inputReader, "shark>") + } + + override def afterAll() { + process.destroy() + process.waitFor() + } + + test("simple select") { + val dataFilePath = TestUtils.dataFilePath + "/kv1.txt" + executeQuery("create table shark_test1(key int, val string);") + executeQuery("load data local inpath '" + dataFilePath+ "' overwrite into table shark_test1;") + executeQuery("""create table shark_test1_cached TBLPROPERTIES ("shark.cache" = "true") as + select * from shark_test1;""") + val out = executeQuery("select * from shark_test1_cached where key = 407;") + assert(out.contains("val_407")) + } + +} diff --git a/src/main/test/scala/shark/ColumnStatsSQLSuite.scala b/src/main/test/scala/shark/ColumnStatsSQLSuite.scala new file mode 100755 index 00000000..705496f2 --- /dev/null +++ b/src/main/test/scala/shark/ColumnStatsSQLSuite.scala @@ -0,0 +1,129 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark + +import org.apache.hadoop.io.BytesWritable + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME + +import org.apache.spark.rdd.RDD + +import shark.memstore2.MemoryMetadataManager + + +class ColumnStatsSQLSuite extends FunSuite with BeforeAndAfterAll { + + val sc: SharkContext = SharkRunner.init() + val sharkMetastore = SharkEnv.memoryMetadataManager + + // import expectSql() shortcut methods + import shark.SharkRunner._ + + override def beforeAll() { + sc.runSql("drop table if exists srcpart_cached") + sc.runSql("create table srcpart_cached(key int, val string) partitioned by (keypart int)") + sc.runSql("""load data local inpath '${hiveconf:shark.test.data.path}/kv1.txt' + into table srcpart_cached partition (keypart = 1)""") + } + + override def afterAll() { + sc.runSql("drop table if exists srcpart_cached") + } + + test("Hive partition stats are tracked") { + val tableOpt = sharkMetastore.getPartitionedTable(DEFAULT_DATABASE_NAME, "srcpart_cached") + assert(tableOpt.isDefined) + val partitionToStatsOpt = tableOpt.get.getStats("keypart=1") + assert(partitionToStatsOpt.isDefined) + val partitionToStats = partitionToStatsOpt.get + // The 'kv1.txt' file loaded into 'keypart=1' in beforeAll() has 2 partitions. + assert(partitionToStats.size == 2) + } + + test("Hive partition stats are tracked after LOADs and INSERTs") { + // Load more data into srcpart_cached + sc.runSql("""load data local inpath '${hiveconf:shark.test.data.path}/kv1.txt' + into table srcpart_cached partition (keypart = 1)""") + val tableOpt = sharkMetastore.getPartitionedTable(DEFAULT_DATABASE_NAME, "srcpart_cached") + assert(tableOpt.isDefined) + var partitionToStatsOpt = tableOpt.get.getStats("keypart=1") + assert(partitionToStatsOpt.isDefined) + var partitionToStats = partitionToStatsOpt.get + // The 'kv1.txt' file loaded into 'keypart=1' has 2 partitions. We've loaded it twice at this + // point. + assert(partitionToStats.size == 4) + + // Append using INSERT command + sc.runSql("insert into table srcpart_cached partition(keypart = 1) select * from test") + partitionToStatsOpt = tableOpt.get.getStats("keypart=1") + assert(partitionToStatsOpt.isDefined) + partitionToStats = partitionToStatsOpt.get + assert(partitionToStats.size == 6) + + // INSERT OVERWRITE should overrwritie old table stats. This also restores srcpart_cached + // to contents contained before this test. + sc.runSql("""insert overwrite table srcpart_cached partition(keypart = 1) + select * from test""") + partitionToStatsOpt = tableOpt.get.getStats("keypart=1") + assert(partitionToStatsOpt.isDefined) + partitionToStats = partitionToStatsOpt.get + assert(partitionToStats.size == 2) + } + + ////////////////////////////////////////////////////////////////////////////// + // End-to-end sanity checks + ////////////////////////////////////////////////////////////////////////////// + test("column pruning filters") { + expectSql("select count(*) from test_cached where key > -1", "500") + } + + test("column pruning group by") { + expectSql("select key, count(*) from test_cached group by key order by key limit 1", "0\t3") + } + + test("column pruning group by with single filter") { + expectSql("select key, count(*) from test_cached where val='val_484' group by key", "484\t1") + } + + test("column pruning aggregate function") { + expectSql("select val, sum(key) from test_cached group by val order by val desc limit 1", + "val_98\t196") + } + + test("column pruning filters for a Hive partition") { + expectSql("select count(*) from srcpart_cached where key > -1", "500") + expectSql("select count(*) from srcpart_cached where key > -1 and keypart = 1", "500") + } + + test("column pruning group by for a Hive partition") { + expectSql("select key, count(*) from srcpart_cached group by key order by key limit 1", "0\t3") + } + + test("column pruning group by with single filter for a Hive partition") { + expectSql("select key, count(*) from srcpart_cached where val='val_484' group by key", "484\t1") + } + + test("column pruning aggregate function for a Hive partition") { + expectSql("select val, sum(key) from srcpart_cached group by val order by val desc limit 1", + "val_98\t196") + } + +} diff --git a/src/main/test/scala/shark/CountDistinctRewriteSuite.scala b/src/main/test/scala/shark/CountDistinctRewriteSuite.scala new file mode 100755 index 00000000..04ef2f55 --- /dev/null +++ b/src/main/test/scala/shark/CountDistinctRewriteSuite.scala @@ -0,0 +1,283 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark + +import java.util.{List => JavaList} + +import scala.collection.JavaConversions._ + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import org.apache.hadoop.hive.ql.lib.Node +import org.apache.hadoop.hive.ql.parse.{ASTNode, HiveParser} +import org.apache.hadoop.hive.ql.parse.{ParseDriver, ParseUtils} + +import shark.api.QueryExecutionException +import shark.parse.ASTRewriteUtil +import shark.parse.ASTRewriteUtil._ + + +class CountDistinctRewriteSuite extends FunSuite with BeforeAndAfterAll { + + /** + * Throws an error if this is not equal to other. + * + * Right now this function only checks the name, type, text and children of the node + * for equality. + */ + def checkEquals(node: ASTNode, other: ASTNode) { + def check(field: String, func: ASTNode => Any) { + assert(func(node) == func(other), + "node: \n" + printTree(node) + "\nis not equal to: \n" + printTree(other) + + "\n for field: " + field) + } + + check("name", _.getName) + check("type", _.getType) + check("text", _.getText) + check("numChildren", (node: ASTNode) => getChildren(node).size) + + val leftChildren = getChildren(node) + val rightChildren = getChildren(other) + leftChildren.zip(rightChildren).foreach { + case (l,r) => checkEquals(l, r) + } + } + + def genAST(command: String): ASTNode = { + try { + ParseUtils.findRootNonNullToken((new ParseDriver()).parse(command)) + } catch { + case e: Exception => + throw new RuntimeException("Failed to parse: " + command) + } + } + + test("Count distinct, single column") { + val command = genAST("select count(distinct key) from src") + val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) + val expectedRewrite = genAST("select count(*) from (select distinct key from src) %s" + .format(ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "0")) + checkEquals(rewrite, expectedRewrite) + } + + test("Count distinct, multiple columns") { + val command = genAST("select count(distinct key, value) from src") + val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) + val expectedRewrite = genAST("select count(*) from (select distinct key, value from src) %s" + .format(ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "0")) + checkEquals(rewrite, expectedRewrite) + } + + test("Multiple columns with expressions") { + val command = genAST("select count(distinct key * 10 - 3, substr(value, 5)) from src") + val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) + val expectedRewrite = genAST( + "select count(*) from (select distinct key * 10 - 3, substr(value, 5) from src) %s" + .format(ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "0")) + checkEquals(rewrite, expectedRewrite) + } + + test("Distinct function outputs") { + val command = genAST("select count(distinct substr(val, 5)) from src") + val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) + val expectedRewrite = genAST(""" + select count(*) from (select distinct substr(val, 5) from src) %s""" + .format(ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "0")) + checkEquals(rewrite, expectedRewrite) + } + + test("Constants aside COUNT DISTINCT in SELECT list") { + val command1 = genAST("select 1, 2, count(distinct key) from src") + val rewrite1 = ASTRewriteUtil.countDistinctToGroupBy(command1) + val expectedRewrite1 = genAST(""" + select 1, 2, count(*) from (select distinct key from src) %s""" + .format(ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "0")) + checkEquals(rewrite1, expectedRewrite1) + + val command2 = genAST("select 1, count(distinct key), 2, 3 from src") + val rewrite2 = ASTRewriteUtil.countDistinctToGroupBy(command2) + val expectedRewrite2 = genAST(""" + select 1, count(*), 2, 3 from (select distinct key from src) %s""" + .format(ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "0")) + checkEquals(rewrite2, expectedRewrite2) + } + + test("COUNT DISTINCT as part of an expression") { + val command = genAST("select count(distinct key) + 10 from src") + val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) + val expectedRewrite = genAST(""" + select count(*) + 10 from (select distinct key from src) %s""" + .format(ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "0")) + checkEquals(rewrite, expectedRewrite) + } + + test("COUNT DISTINCT as part of a subquery") { + val command = genAST("select * from (select count(distinct key) + 10 from src) numDistincts") + val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) + val expectedRewrite = genAST(""" + select * from + (select count(*) + 10 from + (select distinct key from src) %s) numDistincts + """.format(ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "1")) + checkEquals(rewrite, expectedRewrite) + } + + test("COUNT DISTINCT from results of a subquery") { + val command = genAST(""" + select count(distinct a.val) from + (select * from src where key is null) a + join + (select * from src where key is null) b on a.key = b.key + """) + val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) + val expectedRewrite = genAST(""" + select count(*) from + (select distinct a.val from + (select * from src where key is null) a + join + (select * from src where key is null) b on a.key = b.key + ) %s + """.format(ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "0")) + checkEquals(rewrite, expectedRewrite) + } + + test("COUNT DISTINCT from the results of a subquery, as part of an outer subquery") { + val command = genAST(""" + select * from ( + select count(distinct a.val) from + (select * from src where key is null) a + join + (select * from src where key is null) b on a.key = b.key + ) numDistincts + """) + val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) + val expectedRewrite = genAST(""" + select * from + (select count(*) from + (select distinct a.val from + (select * from src where key is null) a + join + (select * from src where key is null) b on a.key = b.key + ) %s + ) numDistincts + """.format(ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "1")) + checkEquals(rewrite, expectedRewrite) + } + + test("Union multiple count distincts") { + val command = genAST(""" + select * from ( + select count(distinct key) from src + union all + select count(distinct value) from src + union all + select count(distinct key) from src1 + union all + select count(distinct value) from src2 + ) distinctKVs""") + val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) + val expectedRewrite = genAST(""" + select * from ( + select count(*) from (select distinct key from src) %s + union all + select count(*) from (select distinct value from src) %s + union all + select count(*) from (select distinct key from src1) %s + union all + select count(*) from (select distinct value from src2) %s + ) distinctKVs""".format( + ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "3", + ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "4", + ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "2", + ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "1")) + checkEquals(rewrite, expectedRewrite) + } + + test("Union multiple count distincts, both over subqueries") { + val command = genAST(""" + select * from ( + select count(distinct a.key) from + (select * from src where key is null) a + join + (select * from src where key is null) b on a.key = b.key + union all + select count(distinct c.value) from + (select * from src where value is null) c + join + (select * from src where value is null) d on c.value = d.value + ) distinctKVs""") + val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) + val expectedRewrite = genAST(""" + select * from ( + select count(*) from + (select distinct a.key from + (select * from src where key is null) a + join + (select * from src where key is null) b on a.key = b.key + ) %s + union all + select count(*) from + (select distinct c.value from + (select * from src where value is null) c + join + (select * from src where value is null) d on c.value = d.value + ) %s + ) distinctKVs""".format( + ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "1", + ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "2")) + checkEquals(rewrite, expectedRewrite) + } + + test("Multiple COUNT DISTINCT in SELECT expression list isn't rewritten (or supported yet)") { + val command = genAST(""" + select + sum(key), + count(distinct key), + count(distinct value) + from src""") + val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) + checkEquals(command, rewrite) + } + + test("COUNT DISTINCT with partitioning key isn't rewritten") { + val command = genAST("select key, count(distinct value) from src group by key") + val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) + checkEquals(command, rewrite) + } + + test("COUNT DISTINCT with LIMIT isn't rewritten") { + val command = genAST("select key, count(distinct value) from src limit 10") + val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) + checkEquals(command, rewrite) + } + + test("COUNT DISTINCT with CUBE and GROUP BY isn't rewritten") { + val command = genAST("select key, count(distinct value) from src group by key with cube") + val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) + checkEquals(command, rewrite) + } + + test("COUNT DISTINCT with ROLLUP and GROUP BY isn't rewritten") { + val command = genAST("select key, count(distinct value) from src group by key with rollup") + val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) + checkEquals(command, rewrite) + } + +} diff --git a/src/main/test/scala/shark/ReflectionSuite.scala b/src/main/test/scala/shark/ReflectionSuite.scala new file mode 100755 index 00000000..b2445505 --- /dev/null +++ b/src/main/test/scala/shark/ReflectionSuite.scala @@ -0,0 +1,112 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark + +import org.scalatest.FunSuite + + +/** + * A suite of test to ensure reflections are used properly in Shark to invoke + * Hive non-public methods. This is needed because we cannot detect reflection + * errors until runtime. Every time reflection is used to expand visibility of + * methods or variables, a test should be added. + */ +class ReflectionSuite extends FunSuite { + + test("CliDriver") { + val c = classOf[org.apache.hadoop.hive.cli.CliDriver] + var m = c.getDeclaredMethod("getFormattedDb", + classOf[org.apache.hadoop.hive.conf.HiveConf], + classOf[org.apache.hadoop.hive.cli.CliSessionState]) + m.setAccessible(true) + assert(m.getReturnType == classOf[String]) + + m = c.getDeclaredMethod( + "spacesForString", classOf[String]) + m.setAccessible(true) + assert(m.getReturnType == classOf[String]) + } + + test("Driver") { + val c = classOf[org.apache.hadoop.hive.ql.Driver] + + var m = c.getDeclaredMethod( + "doAuthorization", classOf[org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer]) + m.setAccessible(true) + assert(m.getReturnType === Void.TYPE) + + m = c.getDeclaredMethod("getHooks", + classOf[org.apache.hadoop.hive.conf.HiveConf.ConfVars], classOf[Class[_]]) + m.setAccessible(true) + assert(m.getReturnType === classOf[java.util.List[_]]) + + var f = c.getDeclaredField("plan") + f.setAccessible(true) + assert(f.getType === classOf[org.apache.hadoop.hive.ql.QueryPlan]) + + f = c.getDeclaredField("ctx") + f.setAccessible(true) + assert(f.getType === classOf[org.apache.hadoop.hive.ql.Context]) + + f = c.getDeclaredField("schema") + f.setAccessible(true) + assert(f.getType === classOf[org.apache.hadoop.hive.metastore.api.Schema]) + + f = c.getDeclaredField("LOG") + f.setAccessible(true) + assert(f.getType === classOf[org.apache.commons.logging.Log]) + } + + test("SemanticAnalyzer") { + val c = classOf[org.apache.hadoop.hive.ql.parse.SemanticAnalyzer] + var m = c.getDeclaredMethod( + "validateCreateView", + classOf[org.apache.hadoop.hive.ql.plan.CreateViewDesc]) + m.setAccessible(true) + assert(m.getReturnType === Void.TYPE) + + m = c.getDeclaredMethod( + "convertRowSchemaToViewSchema", + classOf[org.apache.hadoop.hive.ql.parse.RowResolver]) + m.setAccessible(true) + assert(m.getReturnType === classOf[java.util.List[_]]) + + val f = c.getDeclaredField("viewsExpanded") + f.setAccessible(true) + assert(f.getType === classOf[java.util.ArrayList[_]]) + } + + test("UnionOperator") { + val c = classOf[org.apache.hadoop.hive.ql.exec.UnionOperator] + var f = c.getDeclaredField("needsTransform") + f.setAccessible(true) + assert(f.getType === classOf[Array[Boolean]]) + } + + test("FileSinkOperator") { + val fileSinkCls = classOf[org.apache.hadoop.hive.ql.exec.FileSinkOperator] + var f = fileSinkCls.getDeclaredField("fsp") + f.setAccessible(true) + assert(f.getType === classOf[org.apache.hadoop.hive.ql.exec.FileSinkOperator#FSPaths]) + + val fspCls = classOf[org.apache.hadoop.hive.ql.exec.FileSinkOperator#FSPaths] + f = fspCls.getDeclaredField("finalPaths") + f.setAccessible(true) + assert(f.getType === classOf[Array[org.apache.hadoop.fs.Path]]) + } +} diff --git a/src/main/test/scala/shark/SQLSuite.scala b/src/main/test/scala/shark/SQLSuite.scala new file mode 100755 index 00000000..8175c308 --- /dev/null +++ b/src/main/test/scala/shark/SQLSuite.scala @@ -0,0 +1,1154 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark + +import scala.collection.JavaConversions._ + +import org.scalatest.FunSuite + +import org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME +import org.apache.hadoop.hive.ql.metadata.Hive +import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.UnionRDD +import org.apache.spark.storage.StorageLevel + +import shark.api.QueryExecutionException +import shark.memstore2.{CacheType, MemoryMetadataManager, PartitionedMemoryTable} +import shark.tgf.{RDDSchema, Schema} +// import expectSql() shortcut methods +import shark.SharkRunner._ + + +class SQLSuite extends FunSuite { + + val DEFAULT_DB_NAME = DEFAULT_DATABASE_NAME + val KV1_TXT_PATH = "${hiveconf:shark.test.data.path}/kv1.txt" + + var sc: SharkContext = SharkRunner.init() + var sharkMetastore: MemoryMetadataManager = SharkEnv.memoryMetadataManager + + private def createCachedPartitionedTable( + tableName: String, + numPartitionsToCreate: Int, + maxCacheSize: Int = 10, + cachePolicyClassName: String = "shark.memstore2.LRUCachePolicy" + ): PartitionedMemoryTable = { + sc.runSql("drop table if exists %s".format(tableName)) + sc.runSql(""" + create table %s(key int, value string) + partitioned by (keypart int) + tblproperties('shark.cache' = 'true', + 'shark.cache.policy.maxSize' = '%d', + 'shark.cache.policy' = '%s') + """.format( + tableName, + maxCacheSize, + cachePolicyClassName)) + var partitionNum = 1 + while (partitionNum <= numPartitionsToCreate) { + sc.runSql("""insert into table %s partition(keypart = %d) + select * from test_cached""".format(tableName, partitionNum)) + partitionNum += 1 + } + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( + DEFAULT_DB_NAME, tableName).get + partitionedTable + } + + def isFlattenedUnionRDD(unionRDD: UnionRDD[_]) = { + unionRDD.rdds.find(_.isInstanceOf[UnionRDD[_]]).isEmpty + } + + // Takes a sum over the table's 'key' column, for both the cached contents and the copy on disk. + def expectUnifiedKVTable( + cachedTableName: String, + partSpecOpt: Option[Map[String, String]] = None) { + // Check that the table is in memory and is a unified view. + val sharkTableOpt = sharkMetastore.getTable(DEFAULT_DB_NAME, cachedTableName) + assert(sharkTableOpt.isDefined, "Table %s cannot be found in the Shark metastore") + assert(sharkTableOpt.get.cacheMode == CacheType.MEMORY, + "'shark.cache' field for table %s is not CacheType.MEMORY") + + // Load a non-cached copy of the table into memory. + val cacheSum = sc.sql("select sum(key) from %s".format(cachedTableName))(0) + val hiveTable = Hive.get().getTable(DEFAULT_DB_NAME, cachedTableName) + val location = partSpecOpt match { + case Some(partSpec) => { + val partition = Hive.get().getPartition(hiveTable, partSpec, false /* forceCreate */) + partition.getDataLocation.toString + } + case None => hiveTable.getDataLocation.toString + } + // Create a table with contents loaded from the table's data directory. + val diskTableName = "%s_disk_copy".format(cachedTableName) + sc.sql("drop table if exists %s".format(diskTableName)) + sc.sql("create table %s (key int, value string)".format(diskTableName)) + sc.sql("load data local inpath '%s' into table %s".format(location, diskTableName)) + val diskSum = sc.sql("select sum(key) from %s".format(diskTableName))(0) + assert(diskSum == cacheSum, "Sum of keys from cached and disk contents differ") + } + + ////////////////////////////////////////////////////////////////////////////// + // basic SQL + ////////////////////////////////////////////////////////////////////////////// + test("count") { + expectSql("select count(*) from test", "500") + expectSql("select count(*) from test_cached", "500") + } + + test("filter") { + expectSql("select * from test where key=100 or key=497", + Array("100\tval_100", "100\tval_100", "497\tval_497")) + expectSql("select * from test_cached where key=100 or key=497", + Array("100\tval_100", "100\tval_100", "497\tval_497")) + } + + test("count distinct") { + sc.runSql("set mapred.reduce.tasks=3") + expectSql("select count(distinct key) from test", "309") + expectSql("select count(distinct key) from test_cached", "309") + expectSql( + """|SELECT substr(key,1,1), count(DISTINCT substr(val,5)) from test + |GROUP BY substr(key,1,1)""".stripMargin, + Array("0\t1", "1\t71", "2\t69", "3\t62", "4\t74", "5\t6", "6\t5", "7\t6", "8\t8", "9\t7")) + } + + test("count bigint") { + sc.runSql("drop table if exists test_bigint") + sc.runSql("create table test_bigint (key bigint, val string)") + sc.runSql("""load data local inpath '${hiveconf:shark.test.data.path}/kv1.txt' + OVERWRITE INTO TABLE test_bigint""") + sc.runSql("drop table if exists test_bigint_cached") + sc.runSql("create table test_bigint_cached as select * from test_bigint") + expectSql("select val, count(*) from test_bigint_cached where key=484 group by val", + "val_484\t1") + } + + test("limit") { + assert(sc.runSql("select * from test limit 10").results.length === 10) + assert(sc.runSql("select * from test limit 501").results.length === 500) + sc.runSql("drop table if exists test_limit0") + assert(sc.runSql("select * from test limit 0").results.length === 0) + assert(sc.runSql("create table test_limit0 as select * from test limit 0").results.length === 0) + assert(sc.runSql("select * from test_limit0 limit 0").results.length === 0) + assert(sc.runSql("select * from test_limit0 limit 1").results.length === 0) + } + + ////////////////////////////////////////////////////////////////////////////// + // sorting + ////////////////////////////////////////////////////////////////////////////// + + ignore("full order by") { + expectSql("select * from users order by id", Array("1\tA", "2\tB", "3\tA"), sort = false) + expectSql("select * from users order by id desc", Array("3\tA", "2\tB", "1\tA"), sort = false) + expectSql("select * from users order by name, id", Array("1\tA", "3\tA", "2\tB"), sort = false) + expectSql("select * from users order by name desc, id desc", Array("2\tB", "3\tA", "1\tA"), + sort = false) + } + + test("full order by with limit") { + expectSql("select * from users order by id limit 2", Array("1\tA", "2\tB"), sort = false) + expectSql("select * from users order by id desc limit 2", Array("3\tA", "2\tB"), sort = false) + expectSql("select * from users order by name, id limit 2", Array("1\tA", "3\tA"), sort = false) + expectSql("select * from users order by name desc, id desc limit 2", Array("2\tB", "3\tA"), + sort = false) + } + + ////////////////////////////////////////////////////////////////////////////// + // join + ////////////////////////////////////////////////////////////////////////////// + test("join ouput rows of stand objects") { + assert( + sc.sql("select test1val from users join test1 on users.id=test1.id and users.id=1").head === + "[0,1,2]") + } + + ////////////////////////////////////////////////////////////////////////////// + // map join + ////////////////////////////////////////////////////////////////////////////// + test("map join") { + expectSql("""select u.name, count(c.click) from clicks c join users u on (c.id = u.id) + group by u.name having u.name='A'""", + "A\t3") + } + + test("map join2") { + expectSql("select count(*) from clicks join users on (clicks.id = users.id)", "5") + } + + ////////////////////////////////////////////////////////////////////////////// + // join + ////////////////////////////////////////////////////////////////////////////// + test("outer join on null key") { + expectSql("""select count(distinct a.val) from + (select * from test_null where key is null) a + left outer join + (select * from test_null where key is null) b on a.key=b.key""", "7") + } + + ////////////////////////////////////////////////////////////////////////////// + // cache DDL + ////////////////////////////////////////////////////////////////////////////// + test("Use regular CREATE TABLE and '_cached' suffix to create cached table") { + sc.runSql("drop table if exists empty_table_cached") + sc.runSql("create table empty_table_cached(key string, value string)") + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, "empty_table_cached")) + assert(!SharkEnv.memoryMetadataManager.isHivePartitioned(DEFAULT_DB_NAME, "empty_table_cached")) + } + + test("Use regular CREATE TABLE and table properties to create cached table") { + sc.runSql("drop table if exists empty_table_cached_tbl_props") + sc.runSql("""create table empty_table_cached_tbl_props(key string, value string) + TBLPROPERTIES('shark.cache' = 'true')""") + assert(SharkEnv.memoryMetadataManager.containsTable( + DEFAULT_DB_NAME, "empty_table_cached_tbl_props")) + assert(!SharkEnv.memoryMetadataManager.isHivePartitioned( + DEFAULT_DB_NAME, "empty_table_cached_tbl_props")) + } + + test("Insert into empty cached table") { + sc.runSql("drop table if exists new_table_cached") + sc.runSql("create table new_table_cached(key string, value string)") + sc.runSql("insert into table new_table_cached select * from test where key > -1 limit 499") + expectSql("select count(*) from new_table_cached", "499") + } + + test("rename cached table") { + sc.runSql("drop table if exists test_oldname_cached") + sc.runSql("drop table if exists test_rename") + sc.runSql("create table test_oldname_cached as select * from test") + sc.runSql("alter table test_oldname_cached rename to test_rename") + + assert(!SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, "test_oldname_cached")) + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, "test_rename")) + + expectSql("select count(*) from test_rename", "500") + } + + test("insert into cached tables") { + sc.runSql("drop table if exists test1_cached") + sc.runSql("create table test1_cached as select * from test") + expectSql("select count(*) from test1_cached", "500") + sc.runSql("insert into table test1_cached select * from test where key > -1 limit 499") + expectSql("select count(*) from test1_cached", "999") + } + + test("insert overwrite") { + sc.runSql("drop table if exists test2_cached") + sc.runSql("create table test2_cached as select * from test") + expectSql("select count(*) from test2_cached", "500") + sc.runSql("insert overwrite table test2_cached select * from test where key > -1 limit 499") + expectSql("select count(*) from test2_cached", "499") + } + + test("error when attempting to update cached table(s) using command with multiple INSERTs") { + sc.runSql("drop table if exists multi_insert_test") + sc.runSql("drop table if exists multi_insert_test_cached") + sc.runSql("create table multi_insert_test as select * from test") + sc.runSql("create table multi_insert_test_cached as select * from test") + intercept[QueryExecutionException] { + sc.runSql("""from test + insert into table multi_insert_test select * + insert into table multi_insert_test_cached select *""") + } + } + + test("create cached table with 'shark.cache' flag in table properties") { + sc.runSql("drop table if exists ctas_tbl_props") + sc.runSql("""create table ctas_tbl_props TBLPROPERTIES ('shark.cache'='true') as + select * from test""") + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, "ctas_tbl_props")) + expectSql("select * from ctas_tbl_props where key=407", "407\tval_407") + } + + test("default to Hive table creation when 'shark.cache' flag is false in table properties") { + sc.runSql("drop table if exists ctas_tbl_props_should_not_be_cached") + sc.runSql(""" + CREATE TABLE ctas_tbl_props_result_should_not_be_cached + TBLPROPERTIES ('shark.cache'='false') + AS select * from test""") + assert(!SharkEnv.memoryMetadataManager.containsTable( + DEFAULT_DB_NAME, "ctas_tbl_props_should_not_be_cached")) + } + + test("cached tables with complex types") { + sc.runSql("drop table if exists test_complex_types") + sc.runSql("drop table if exists test_complex_types_cached") + sc.runSql("""CREATE TABLE test_complex_types ( + a STRING, b ARRAY, c ARRAY>, d MAP>)""") + sc.runSql("""load data local inpath '${hiveconf:shark.test.data.path}/create_nested_type.txt' + overwrite into table test_complex_types""") + sc.runSql("""create table test_complex_types_cached TBLPROPERTIES ("shark.cache" = "true") as + select * from test_complex_types""") + + assert(sc.sql("select a from test_complex_types_cached where a = 'a0'").head === "a0") + + assert(sc.sql("select b from test_complex_types_cached where a = 'a0'").head === + """["b00","b01"]""") + + assert(sc.sql("select c from test_complex_types_cached where a = 'a0'").head === + """[{"c001":"C001","c002":"C002"},{"c011":null,"c012":"C012"}]""") + + assert(sc.sql("select d from test_complex_types_cached where a = 'a0'").head === + """{"d01":["d011","d012"],"d02":["d021","d022"]}""") + + assert(SharkEnv.memoryMetadataManager.containsTable( + DEFAULT_DB_NAME, "test_complex_types_cached")) + } + + test("disable caching by default") { + sc.runSql("set shark.cache.flag.checkTableName=false") + sc.runSql("drop table if exists should_not_be_cached") + sc.runSql("create table should_not_be_cached as select * from test") + expectSql("select key from should_not_be_cached where key = 407", "407") + assert(!SharkEnv.memoryMetadataManager.containsTable( + DEFAULT_DB_NAME, "should_not_be_cached")) + sc.runSql("set shark.cache.flag.checkTableName=true") + } + + test("cached table name should be case-insensitive") { + sc.runSql("drop table if exists sharkTest5Cached") + sc.runSql("""create table sharkTest5Cached TBLPROPERTIES ("shark.cache" = "true") as + select * from test""") + expectSql("select val from sharktest5Cached where key = 407", "val_407") + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, "sharkTest5Cached")) + } + + test("dropping cached tables should clean up RDDs") { + sc.runSql("drop table if exists sharkTest5Cached") + sc.runSql("""create table sharkTest5Cached TBLPROPERTIES ("shark.cache" = "true") as + select * from test""") + sc.runSql("drop table sharkTest5Cached") + assert(!SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, "sharkTest5Cached")) + } + + test("lateral view explode column pruning") { + // If column pruner doesn't take lateral view into account, the first result will be null. + assert(sc.runSql("""select * from test_cached + lateral view explode(array(1, 2, 3)) exploadedTbl as col1""").results.head.head != null) + } + + ////////////////////////////////////////////////////////////////////////////// + // Caching Hive-partititioned tables + // Note: references to 'partition' for this section refer to a Hive-partition. + ////////////////////////////////////////////////////////////////////////////// + test("Use regular CREATE TABLE and '_cached' suffix to create cached, partitioned table") { + sc.runSql("drop table if exists empty_part_table_cached") + sc.runSql("""create table empty_part_table_cached(key int, value string) + partitioned by (keypart int)""") + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, "empty_part_table_cached")) + assert(SharkEnv.memoryMetadataManager.isHivePartitioned( + DEFAULT_DB_NAME, "empty_part_table_cached")) + } + + test("Use regular CREATE TABLE and table properties to create cached, partitioned table") { + sc.runSql("drop table if exists empty_part_table_cached_tbl_props") + sc.runSql("""create table empty_part_table_cached_tbl_props(key int, value string) + partitioned by (keypart int) tblproperties('shark.cache' = 'true')""") + assert(SharkEnv.memoryMetadataManager.containsTable( + DEFAULT_DB_NAME, "empty_part_table_cached_tbl_props")) + assert(SharkEnv.memoryMetadataManager.isHivePartitioned( + DEFAULT_DB_NAME, "empty_part_table_cached_tbl_props")) + } + + test("alter cached table by adding a new partition") { + sc.runSql("drop table if exists alter_part_cached") + sc.runSql("""create table alter_part_cached(key int, value string) + partitioned by (keypart int)""") + sc.runSql("""alter table alter_part_cached add partition(keypart = 1)""") + val tableName = "alter_part_cached" + val partitionColumn = "keypart=1" + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( + DEFAULT_DB_NAME, tableName).get + assert(partitionedTable.containsPartition(partitionColumn)) + } + + test("alter cached table by dropping a partition") { + sc.runSql("drop table if exists alter_drop_part_cached") + sc.runSql("""create table alter_drop_part_cached(key int, value string) + partitioned by (keypart int)""") + sc.runSql("""alter table alter_drop_part_cached add partition(keypart = 1)""") + val tableName = "alter_drop_part_cached" + val partitionColumn = "keypart=1" + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( + DEFAULT_DB_NAME, tableName).get + assert(partitionedTable.containsPartition(partitionColumn)) + sc.runSql("""alter table alter_drop_part_cached drop partition(keypart = 1)""") + assert(!partitionedTable.containsPartition(partitionColumn)) + } + + test("insert into a partition of a cached table") { + val tableName = "insert_part_cached" + val partitionedTable = createCachedPartitionedTable( + tableName, + 1 /* numPartitionsToCreate */) + expectSql("select value from insert_part_cached where key = 407 and keypart = 1", "val_407") + + } + + test("insert overwrite a partition of a cached table") { + val tableName = "insert_over_part_cached" + val partitionedTable = createCachedPartitionedTable( + tableName, + 1 /* numPartitionsToCreate */) + expectSql("""select value from insert_over_part_cached + where key = 407 and keypart = 1""", "val_407") + sc.runSql("""insert overwrite table insert_over_part_cached partition(keypart = 1) + select key, -1 from test""") + expectSql("select value from insert_over_part_cached where key = 407 and keypart = 1", "-1") + } + + test("scan cached, partitioned table that's empty") { + sc.runSql("drop table if exists empty_part_table_cached") + sc.runSql("""create table empty_part_table_cached(key int, value string) + partitioned by (keypart int)""") + expectSql("select count(*) from empty_part_table_cached", "0") + } + + test("scan cached, partitioned table that has a single partition") { + val tableName = "scan_single_part_cached" + val partitionedTable = createCachedPartitionedTable( + tableName, + 1 /* numPartitionsToCreate */) + expectSql("select * from scan_single_part_cached where key = 407", "407\tval_407\t1") + } + + test("scan cached, partitioned table that has multiple partitions") { + val tableName = "scan_mult_part_cached" + val partitionedTable = createCachedPartitionedTable( + tableName, + 3 /* numPartitionsToCreate */) + expectSql("select * from scan_mult_part_cached where key = 407 order by keypart", + Array("407\tval_407\t1", "407\tval_407\t2", "407\tval_407\t3")) + } + + test("drop/unpersist cached, partitioned table that has multiple partitions") { + val tableName = "drop_mult_part_cached" + val partitionedTable = createCachedPartitionedTable( + tableName, + 3 /* numPartitionsToCreate */) + val keypart1RDD = partitionedTable.getPartition("keypart=1") + val keypart2RDD = partitionedTable.getPartition("keypart=2") + val keypart3RDD = partitionedTable.getPartition("keypart=3") + sc.runSql("drop table drop_mult_part_cached ") + assert(!SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) + // All RDDs should have been unpersisted. + assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE) + assert(keypart2RDD.get.getStorageLevel == StorageLevel.NONE) + assert(keypart3RDD.get.getStorageLevel == StorageLevel.NONE) + } + + test("drop cached partition represented by a UnionRDD (i.e., the result of multiple inserts)") { + val tableName = "drop_union_part_cached" + val partitionedTable = createCachedPartitionedTable( + tableName, + 1 /* numPartitionsToCreate */) + sc.runSql("insert into table drop_union_part_cached partition(keypart = 1) select * from test") + sc.runSql("insert into table drop_union_part_cached partition(keypart = 1) select * from test") + sc.runSql("insert into table drop_union_part_cached partition(keypart = 1) select * from test") + val keypart1RDD = partitionedTable.getPartition("keypart=1") + sc.runSql("drop table drop_union_part_cached") + assert(!SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) + // All RDDs should have been unpersisted. + assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE) + } + + ////////////////////////////////////////////////////////////////////////////// + // RDD(partition) eviction policy for cached Hive-partititioned tables + ////////////////////////////////////////////////////////////////////////////// + + test("shark.memstore2.CacheAllPolicy is the default policy") { + val tableName = "default_policy_cached" + sc.runSql("""create table default_policy_cached(key int, value string) + partitioned by (keypart int)""") + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( + DEFAULT_DB_NAME, tableName).get + val cachePolicy = partitionedTable.cachePolicy + assert(cachePolicy.isInstanceOf[shark.memstore2.CacheAllPolicy[_, _]]) + } + + test("LRU: RDDs are not evicted if the cache isn't full.") { + val tableName = "evict_partitions_maxSize" + val partitionedTable = createCachedPartitionedTable( + tableName, + 2 /* numPartitionsToCreate */, + 3 /* maxCacheSize */, + "shark.memstore2.LRUCachePolicy") + val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") + assert(TestUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + } + + test("LRU: RDDs are evicted when the max size is reached.") { + val tableName = "evict_partitions_maxSize" + val partitionedTable = createCachedPartitionedTable( + tableName, + 3 /* numPartitionsToCreate */, + 3 /* maxCacheSize */, + "shark.memstore2.LRUCachePolicy") + val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") + assert(TestUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + sc.runSql("""insert into table evict_partitions_maxSize partition(keypart = 4) + select * from test""") + assert(TestUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.NONE) + } + + test("LRU: RDD eviction accounts for partition scans - a cache.get()") { + val tableName = "evict_partitions_with_get" + val partitionedTable = createCachedPartitionedTable( + tableName, + 3 /* numPartitionsToCreate */, + 3 /* maxCacheSize */, + "shark.memstore2.LRUCachePolicy") + val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") + val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") + assert(TestUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + assert(TestUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) + sc.runSql("select count(1) from evict_partitions_with_get where keypart = 1") + sc.runSql("""insert into table evict_partitions_with_get partition(keypart = 4) + select * from test""") + assert(TestUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + + assert(TestUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.NONE) + } + + test("LRU: RDD eviction accounts for INSERT INTO - a cache.get().") { + val tableName = "evict_partitions_insert_into" + val partitionedTable = createCachedPartitionedTable( + tableName, + 3 /* numPartitionsToCreate */, + 3 /* maxCacheSize */, + "shark.memstore2.LRUCachePolicy") + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) + val oldKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") + val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") + assert(TestUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + assert(TestUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) + sc.runSql("""insert into table evict_partitions_insert_into partition(keypart = 1) + select * from test""") + sc.runSql("""insert into table evict_partitions_insert_into partition(keypart = 4) + select * from test""") + assert(TestUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + val newKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") + assert(TestUtils.getStorageLevelOfRDD(newKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + + val keypart2StorageLevel = TestUtils.getStorageLevelOfRDD(keypart2RDD.get) + assert(keypart2StorageLevel == StorageLevel.NONE) + } + + test("LRU: RDD eviction accounts for INSERT OVERWRITE - a cache.put()") { + val tableName = "evict_partitions_insert_overwrite" + val partitionedTable = createCachedPartitionedTable( + tableName, + 3 /* numPartitionsToCreate */, + 3 /* maxCacheSize */, + "shark.memstore2.LRUCachePolicy") + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) + val oldKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") + val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") + assert(TestUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + assert(TestUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) + sc.runSql("""insert overwrite table evict_partitions_insert_overwrite partition(keypart = 1) + select * from test""") + sc.runSql("""insert into table evict_partitions_insert_overwrite partition(keypart = 4) + select * from test""") + assert(TestUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.NONE) + val newKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") + assert(TestUtils.getStorageLevelOfRDD(newKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) + + val keypart2StorageLevel = TestUtils.getStorageLevelOfRDD(keypart2RDD.get) + assert(keypart2StorageLevel == StorageLevel.NONE) + } + + test("LRU: RDD eviction accounts for ALTER TABLE DROP PARTITION - a cache.remove()") { + val tableName = "evict_partitions_removals" + val partitionedTable = createCachedPartitionedTable( + tableName, + 3 /* numPartitionsToCreate */, + 3 /* maxCacheSize */, + "shark.memstore2.LRUCachePolicy") + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) + sc.runSql("alter table evict_partitions_removals drop partition(keypart = 1)") + sc.runSql("""insert into table evict_partitions_removals partition(keypart = 4) + select * from test""") + sc.runSql("""insert into table evict_partitions_removals partition(keypart = 5) + select * from test""") + val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") + assert(TestUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.NONE) + } + + test("LRU: get() reloads an RDD previously unpersist()'d.") { + val tableName = "reload_evicted_partition" + val partitionedTable = createCachedPartitionedTable( + tableName, + 3 /* numPartitionsToCreate */, + 3 /* maxCacheSize */, + "shark.memstore2.LRUCachePolicy") + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) + val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") + val lvl = TestUtils.getStorageLevelOfRDD(keypart1RDD.get) + assert(lvl == StorageLevel.MEMORY_AND_DISK, "got: " + lvl) + sc.runSql("""insert into table reload_evicted_partition partition(keypart = 4) + select * from test""") + assert(TestUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.NONE) + + // Scanning partition (keypart = 1) should reload the corresponding RDD into the cache, and + // cause eviction of the RDD for partition (keypart = 2). + sc.runSql("select count(1) from reload_evicted_partition where keypart = 1") + assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) + val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") + val keypart2StorageLevel = TestUtils.getStorageLevelOfRDD(keypart2RDD.get) + assert(keypart2StorageLevel == StorageLevel.NONE, + "StorageLevel for partition(keypart=2) should be NONE, but got: " + keypart2StorageLevel) + } + + test("FIFO: get() does not reload an RDD previously unpersist()'d.") { + val tableName = "dont_reload_evicted_partition" + val partitionedTable = createCachedPartitionedTable( + tableName, + 3 /* numPartitionsToCreate */, + 3 /* maxCacheSize */, + "shark.memstore2.FIFOCachePolicy") + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) + val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") + val lvl = TestUtils.getStorageLevelOfRDD(keypart1RDD.get) + assert(lvl == StorageLevel.MEMORY_AND_DISK, "got: " + lvl) + sc.runSql("""insert into table dont_reload_evicted_partition partition(keypart = 4) + select * from test""") + assert(TestUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.NONE) + + // Scanning partition (keypart = 1) should reload the corresponding RDD into the cache, and + // cause eviction of the RDD for partition (keypart = 2). + sc.runSql("select count(1) from dont_reload_evicted_partition where keypart = 1") + assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE, "got: " + keypart1RDD.get.getStorageLevel) + } + + /////////////////////////////////////////////////////////////////////////////////////// + // Prevent nested UnionRDDs - those should be "flattened" in MemoryStoreSinkOperator. + /////////////////////////////////////////////////////////////////////////////////////// + + test("flatten UnionRDDs") { + sc.sql("create table flat_cached as select * from test_cached") + sc.sql("insert into table flat_cached select * from test") + val tableName = "flat_cached" + var memoryTable = SharkEnv.memoryMetadataManager.getMemoryTable(DEFAULT_DB_NAME, tableName).get + var unionRDD = memoryTable.getRDD.get.asInstanceOf[UnionRDD[_]] + val numParentRDDs = unionRDD.rdds.size + assert(isFlattenedUnionRDD(unionRDD)) + + // Insert another set of query results. The flattening should kick in here. + sc.sql("insert into table flat_cached select * from test") + unionRDD = memoryTable.getRDD.get.asInstanceOf[UnionRDD[_]] + assert(isFlattenedUnionRDD(unionRDD)) + assert(unionRDD.rdds.size == numParentRDDs + 1) + } + + test("flatten UnionRDDs for partitioned tables") { + sc.sql("drop table if exists part_table_cached") + sc.sql("""create table part_table_cached(key int, value string) + partitioned by (keypart int)""") + sc.sql("alter table part_table_cached add partition(keypart = 1)") + sc.sql("insert into table part_table_cached partition(keypart = 1) select * from flat_cached") + val tableName = "part_table_cached" + val partitionKey = "keypart=1" + val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( + DEFAULT_DB_NAME, tableName).get + var unionRDD = partitionedTable.keyToPartitions.get(partitionKey).get.asInstanceOf[UnionRDD[_]] + val numParentRDDs = unionRDD.rdds.size + assert(isFlattenedUnionRDD(unionRDD)) + + // Insert another set of query results into the same partition. + // The flattening should kick in here. + sc.runSql("insert into table part_table_cached partition(keypart = 1) select * from flat_cached") + unionRDD = partitionedTable.getPartition(partitionKey).get.asInstanceOf[UnionRDD[_]] + assert(isFlattenedUnionRDD(unionRDD)) + assert(unionRDD.rdds.size == numParentRDDs + 1) + } + + ////////////////////////////////////////////////////////////////////////////// + // Tableau bug + ////////////////////////////////////////////////////////////////////////////// + + test("tableau bug / adw") { + sc.sql("drop table if exists adw") + sc.sql("""create table adw TBLPROPERTIES ("shark.cache" = "true") as + select cast(key as int) as k, val from test""") + expectSql("select count(k) from adw where val='val_487' group by 1 having count(1) > 0", "1") + } + + ////////////////////////////////////////////////////////////////////////////// + // Partition pruning + ////////////////////////////////////////////////////////////////////////////// + + test("sel star pruning") { + sc.sql("drop table if exists selstar") + sc.sql("""create table selstar TBLPROPERTIES ("shark.cache" = "true") as + select * from test""") + expectSql("select * from selstar where val='val_487'","487 val_487") + } + + test("map pruning with functions in between clause") { + sc.sql("drop table if exists mapsplitfunc") + sc.sql("drop table if exists mapsplitfunc_cached") + sc.sql("create table mapsplitfunc(k bigint, v string)") + sc.sql("""load data local inpath '${hiveconf:shark.test.data.path}/kv1.txt' + OVERWRITE INTO TABLE mapsplitfunc""") + sc.sql("create table mapsplitfunc_cached as select * from mapsplitfunc") + expectSql("""select count(*) from mapsplitfunc_cached + where month(from_unixtime(k)) between "1" and "12" """, Array[String]("500")) + expectSql("""select count(*) from mapsplitfunc_cached + where year(from_unixtime(k)) between "2013" and "2014" """, Array[String]("0")) + } + + test("map pruning with functions in in clause") { + expectSql("""select count(*) from mapsplitfunc_cached + where year(from_unixtime(k)) in ("2013", concat("201", "4")) """, Array[String]("0")) + } + + ////////////////////////////////////////////////////////////////////////////// + // SharkContext APIs (e.g. sql2rdd, sql) + ////////////////////////////////////////////////////////////////////////////// + + test("cached table in different new database") { + sc.sql("drop table if exists selstar") + sc.sql("""create table selstar TBLPROPERTIES ("shark.cache" = "true") as + select * from default.test """) + sc.sql("use seconddb") + sc.sql("drop table if exists selstar") + sc.sql("""create table selstar TBLPROPERTIES ("shark.cache" = "true") as + select * from default.test where key != 'val_487' """) + + sc.sql("use default") + expectSql("select * from selstar where val='val_487'","487 val_487") + + assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, "selstar")) + assert(SharkEnv.memoryMetadataManager.containsTable("seconddb", "selstar")) + + } + + ////////////////////////////////////////////////////////////////////////////// + // various data types + ////////////////////////////////////////////////////////////////////////////// + + test("boolean data type") { + sc.sql("drop table if exists checkboolean") + sc.sql("""create table checkboolean TBLPROPERTIES ("shark.cache" = "true") as + select key, val, true as flag from test where key < "300" """) + sc.sql("""insert into table checkboolean + select key, val, false as flag from test where key > "300" """) + expectSql("select flag, count(*) from checkboolean group by flag order by flag asc", + Array[String]("false\t208", "true\t292")) + } + + test("byte data type") { + sc.sql("drop table if exists checkbyte") + sc.sql("drop table if exists checkbyte_cached") + sc.sql("""create table checkbyte (key string, val string, flag tinyint) """) + sc.sql("""insert into table checkbyte + select key, val, 1 from test where key < "300" """) + sc.sql("""insert into table checkbyte + select key, val, 0 from test where key > "300" """) + sc.sql("""create table checkbyte_cached as select * from checkbyte""") + expectSql("select flag, count(*) from checkbyte_cached group by flag order by flag asc", + Array[String]("0\t208", "1\t292")) + } + + test("binary data type") { + + sc.sql("drop table if exists checkbinary") + sc.sql("drop table if exists checkbinary_cached") + sc.sql("""create table checkbinary (key string, flag binary) """) + sc.sql("""insert into table checkbinary + select key, cast(val as binary) as flag from test where key < "300" """) + sc.sql("""insert into table checkbinary + select key, cast(val as binary) as flag from test where key > "300" """) + sc.sql("create table checkbinary_cached as select key, flag from checkbinary") + expectSql("select cast(flag as string) as f from checkbinary_cached order by f asc limit 2", + Array[String]("val_0", "val_0")) + } + + test("short data type") { + sc.sql("drop table if exists checkshort") + sc.sql("drop table if exists checkshort_cached") + sc.sql("""create table checkshort (key string, val string, flag smallint) """) + sc.sql("""insert into table checkshort + select key, val, 23 as flag from test where key < "300" """) + sc.sql("""insert into table checkshort + select key, val, 36 as flag from test where key > "300" """) + sc.sql("create table checkshort_cached as select key, val, flag from checkshort") + expectSql("select flag, count(*) from checkshort_cached group by flag order by flag asc", + Array[String]("23\t292", "36\t208")) + } + + ////////////////////////////////////////////////////////////////////////////// + // SharkContext APIs (e.g. sql2rdd, sql) + ////////////////////////////////////////////////////////////////////////////// + + test("sql max number of rows returned") { + assert(sc.runSql("select * from test").results.size === 500) + assert(sc.runSql("select * from test", 100).results.size === 100) + } + + test("sql2rdd") { + var rdd = sc.sql2rdd("select * from test") + assert(rdd.count === 500) + rdd = sc.sql2rdd("select * from test_cached") + assert(rdd.count === 500) + val collected = rdd.map(r => r.getInt(0)).collect().sortWith(_ < _) + assert(collected(0) === 0) + assert(collected(499) === 498) + assert(collected.size === 500) + } + + test("null values in sql2rdd") { + val nullsRdd = sc.sql2rdd("select * from test_null where key is null") + val nulls = nullsRdd.map(r => r.getInt(0)).collect() + assert(nulls(0) === null) + assert(nulls.size === 10) + } + + test("sql exception") { + val e = intercept[QueryExecutionException] { sc.runSql("asdfasdfasdfasdf") } + e.getMessage.contains("semantic") + } + + test("sql2rdd exception") { + val e = intercept[QueryExecutionException] { sc.sql2rdd("asdfasdfasdfasdf") } + e.getMessage.contains("semantic") + } + + ////////////////////////////////////////////////////////////////////////////// + // Default cache mode is CacheType.MEMORY (unified view) + ////////////////////////////////////////////////////////////////////////////// + test ("Table created by CREATE TABLE, with table properties, is CacheType.MEMORY by default") { + sc.runSql("drop table if exists test_unify_creation") + sc.runSql("""create table test_unify_creation (key int, val string) + tblproperties('shark.cache'='true')""") + val table = sharkMetastore.getTable(DEFAULT_DB_NAME, "test_unify_creation").get + assert(table.cacheMode == CacheType.MEMORY) + sc.runSql("drop table if exists test_unify_creation") + } + + test ("Table created by CREATE TABLE, with '_cached', is CacheType.MEMORY_ONLY by default") { + sc.runSql("drop table if exists test_unify_creation_cached") + sc.runSql("create table test_unify_creation_cached(key int, val string)") + val table = sharkMetastore.getTable(DEFAULT_DB_NAME, "test_unify_creation_cached").get + assert(table.cacheMode == CacheType.MEMORY_ONLY) + sc.runSql("drop table if exists test_unify_creation_cached") + } + + test ("Table created by CTAS, with table properties, is CacheType.MEMORY by default") { + sc.runSql("drop table if exists test_unify_ctas") + sc.runSql("""create table test_unify_ctas tblproperties('shark.cache' = 'true') + as select * from test""") + val table = sharkMetastore.getTable(DEFAULT_DB_NAME, "test_unify_ctas").get + assert(table.cacheMode == CacheType.MEMORY) + expectSql("select count(*) from test_unify_ctas", "500") + sc.runSql("drop table if exists test_unify_ctas") + } + + test ("Table created by CTAS, with '_cached', is CacheType.MEMORY_ONLY by default") { + sc.runSql("drop table if exists test_unify_ctas_cached") + sc.runSql("create table test_unify_ctas_cached as select * from test") + val table = sharkMetastore.getTable(DEFAULT_DB_NAME, "test_unify_ctas_cached").get + assert(table.cacheMode == CacheType.MEMORY_ONLY) + expectSql("select count(*) from test_unify_ctas_cached", "500") + sc.runSql("drop table if exists test_unify_ctas_cached") + } + + test ("CREATE TABLE when 'shark.cache' is CacheType.MEMORY_ONLY") { + sc.runSql("drop table if exists test_non_unify_creation") + sc.runSql("""create table test_non_unify_creation(key int, val string) + tblproperties('shark.cache' = 'memory_only')""") + val table = sharkMetastore.getTable(DEFAULT_DB_NAME, "test_non_unify_creation").get + assert(table.cacheMode == CacheType.MEMORY_ONLY) + sc.runSql("drop table if exists test_non_unify_creation") + } + + test ("CTAS when 'shark.cache' is CacheType.MEMORY_ONLY") { + sc.runSql("drop table if exists test_non_unify_ctas") + sc.runSql("""create table test_non_unify_ctas tblproperties + ('shark.cache' = 'memory_only') as select * from test""") + val table = sharkMetastore.getTable(DEFAULT_DB_NAME, "test_non_unify_ctas").get + assert(table.cacheMode == CacheType.MEMORY_ONLY) + sc.runSql("drop table if exists test_non_unify_ctas") + } + + ////////////////////////////////////////////////////////////////////////////// + // LOAD for tables cached in memory and stored on disk (unified view) + ////////////////////////////////////////////////////////////////////////////// + test ("LOAD INTO unified view") { + sc.runSql("drop table if exists unified_view_cached") + sc.runSql( + """create table unified_view_cached (key int, value string) + |tblproperties("shark.cache" = "memory") + """.stripMargin) + sc.runSql("load data local inpath '%s' into table unified_view_cached".format(KV1_TXT_PATH)) + expectUnifiedKVTable("unified_view_cached") + expectSql("select count(*) from unified_view_cached", "500") + sc.runSql("drop table if exists unified_view_cached") + } + + test ("LOAD OVERWRITE unified view") { + sc.runSql("drop table if exists unified_overwrite_cached") + sc.runSql("create table unified_overwrite_cached (key int, value string)" + + "tblproperties(\"shark.cache\" = \"memory\")") + sc.runSql("load data local inpath '%s' into table unified_overwrite_cached". + format("${hiveconf:shark.test.data.path}/kv3.txt")) + expectSql("select count(*) from unified_overwrite_cached", "25") + sc.runSql("load data local inpath '%s' overwrite into table unified_overwrite_cached". + format(KV1_TXT_PATH)) + // Make sure the cached contents matches the disk contents. + expectUnifiedKVTable("unified_overwrite_cached") + expectSql("select count(*) from unified_overwrite_cached", "500") + sc.runSql("drop table if exists unified_overwrite_cached") + } + + test ("LOAD INTO partitioned unified view") { + sc.runSql("drop table if exists unified_view_part_cached") + sc.runSql("""create table unified_view_part_cached (key int, value string) + partitioned by (keypart int) tblproperties("shark.cache" = "memory")""") + sc.runSql("""load data local inpath '%s' into table unified_view_part_cached + partition(keypart = 1)""".format(KV1_TXT_PATH)) + expectUnifiedKVTable("unified_view_part_cached", Some(Map("keypart" -> "1"))) + expectSql("select count(*) from unified_view_part_cached", "500") + sc.runSql("drop table if exists unified_view_part_cached") + } + + test ("LOAD OVERWRITE partitioned unified view") { + sc.runSql("drop table if exists unified_overwrite_part_cached") + sc.runSql("""create table unified_overwrite_part_cached (key int, value string) + partitioned by (keypart int) tblproperties("shark.cache" = "memory")""") + sc.runSql("""load data local inpath '%s' overwrite into table unified_overwrite_part_cached + partition(keypart = 1)""".format(KV1_TXT_PATH)) + expectUnifiedKVTable("unified_overwrite_part_cached", Some(Map("keypart" -> "1"))) + expectSql("select count(*) from unified_overwrite_part_cached", "500") + sc.runSql("drop table if exists unified_overwrite_part_cached") + } + + ////////////////////////////////////////////////////////////////////////////// + // INSERT for tables cached in memory and stored on disk (unified view) + ////////////////////////////////////////////////////////////////////////////// + test ("INSERT INTO unified view") { + sc.runSql("drop table if exists unified_view_cached") + sc.runSql("create table unified_view_cached tblproperties('shark.cache'='memory') " + + "as select * from test_cached") + sc.runSql("insert into table unified_view_cached select * from test_cached") + expectUnifiedKVTable("unified_view_cached") + expectSql("select count(*) from unified_view_cached", "1000") + sc.runSql("drop table if exists unified_view_cached") + } + + test ("INSERT OVERWRITE unified view") { + sc.runSql("drop table if exists unified_overwrite_cached") + sc.runSql("create table unified_overwrite_cached tblproperties('shark.cache'='memory')" + + "as select * from test") + sc.runSql("insert overwrite table unified_overwrite_cached select * from test_cached") + expectUnifiedKVTable("unified_overwrite_cached") + expectSql("select count(*) from unified_overwrite_cached", "500") + sc.runSql("drop table if exists unified_overwrite_cached") + } + + test ("INSERT INTO partitioned unified view") { + sc.runSql("drop table if exists unified_view_part_cached") + sc.runSql("""create table unified_view_part_cached (key int, value string) + partitioned by (keypart int) + tblproperties('shark.cache'='memory')""") + sc.runSql("""insert into table unified_view_part_cached partition (keypart = 1) + select * from test_cached""") + expectUnifiedKVTable("unified_view_part_cached", Some(Map("keypart" -> "1"))) + expectSql("select count(*) from unified_view_part_cached where keypart = 1", "500") + sc.runSql("drop table if exists unified_view_part_cached") + } + + test ("INSERT OVERWRITE partitioned unified view") { + sc.runSql("drop table if exists unified_overwrite_part_cached") + sc.runSql("""create table unified_overwrite_part_cached (key int, value string) + partitioned by (keypart int) tblproperties('shark.cache'='memory')""") + sc.runSql("""insert overwrite table unified_overwrite_part_cached partition (keypart = 1) + select * from test_cached""") + expectUnifiedKVTable("unified_overwrite_part_cached", Some(Map("keypart" -> "1"))) + expectSql("select count(*) from unified_overwrite_part_cached", "500") + sc.runSql("drop table if exists unified_overwrite_part_cached") + } + + ////////////////////////////////////////////////////////////////////////////// + // CACHE and ALTER TABLE commands + ////////////////////////////////////////////////////////////////////////////// + test ("ALTER TABLE caches non-partitioned table if 'shark.cache' is set to true") { + sc.runSql("drop table if exists unified_load") + sc.runSql("create table unified_load as select * from test") + sc.runSql("alter table unified_load set tblproperties('shark.cache' = 'true')") + expectUnifiedKVTable("unified_load") + sc.runSql("drop table if exists unified_load") + } + + test ("ALTER TABLE caches partitioned table if 'shark.cache' is set to true") { + sc.runSql("drop table if exists unified_part_load") + sc.runSql("create table unified_part_load (key int, value string) partitioned by (keypart int)") + sc.runSql("insert into table unified_part_load partition (keypart=1) select * from test_cached") + sc.runSql("alter table unified_part_load set tblproperties('shark.cache' = 'true')") + expectUnifiedKVTable("unified_part_load", Some(Map("keypart" -> "1"))) + sc.runSql("drop table if exists unified_part_load") + } + + test ("ALTER TABLE uncaches non-partitioned table if 'shark.cache' is set to false") { + sc.runSql("drop table if exists unified_load") + sc.runSql("create table unified_load as select * from test") + sc.runSql("alter table unified_load set tblproperties('shark.cache' = 'false')") + assert(!sharkMetastore.containsTable(DEFAULT_DB_NAME, "unified_load")) + expectSql("select count(*) from unified_load", "500") + sc.runSql("drop table if exists unified_load") + } + + test ("ALTER TABLE uncaches partitioned table if 'shark.cache' is set to false") { + sc.runSql("drop table if exists unified_part_load") + sc.runSql("create table unified_part_load (key int, value string) partitioned by (keypart int)") + sc.runSql("insert into table unified_part_load partition (keypart=1) select * from test_cached") + sc.runSql("alter table unified_part_load set tblproperties('shark.cache' = 'false')") + assert(!sharkMetastore.containsTable(DEFAULT_DB_NAME, "unified_part_load")) + expectSql("select count(*) from unified_part_load", "500") + sc.runSql("drop table if exists unified_part_load") + } + + test ("UNCACHE behaves like ALTER TABLE SET TBLPROPERTIES ...") { + sc.runSql("drop table if exists unified_load") + sc.runSql("create table unified_load as select * from test") + sc.runSql("cache unified_load") + // Double check the table properties. + val tableName = "unified_load" + val hiveTable = Hive.get().getTable(DEFAULT_DB_NAME, tableName) + assert(hiveTable.getProperty("shark.cache") == "MEMORY") + // Check that the cache and disk contents are synchronized. + expectUnifiedKVTable(tableName) + sc.runSql("drop table if exists unified_load") + } + + test ("CACHE behaves like ALTER TABLE SET TBLPROPERTIES ...") { + sc.runSql("drop table if exists unified_load") + sc.runSql("create table unified_load as select * from test") + sc.runSql("cache unified_load") + // Double check the table properties. + val tableName = "unified_load" + val hiveTable = Hive.get().getTable(DEFAULT_DB_NAME, tableName) + assert(hiveTable.getProperty("shark.cache") == "MEMORY") + // Check that the cache and disk contents are synchronized. + expectUnifiedKVTable(tableName) + sc.runSql("drop table if exists unified_load") + } + + ////////////////////////////////////////////////////////////////////////////// + // Cached table persistence + ////////////////////////////////////////////////////////////////////////////// + ignore ("Cached tables persist across Shark metastore shutdowns.") { + val globalCachedTableNames = Seq("test_cached", "test_null_cached", "clicks_cached", + "users_cached") + + // Number of rows for each cached table. + val cachedTableCounts = new Array[String](globalCachedTableNames.size) + for ((tableName, i) <- globalCachedTableNames.zipWithIndex) { + val hiveTable = Hive.get().getTable(DEFAULT_DB_NAME, tableName) + val cachedCount = sc.sql("select count(*) from %s".format(tableName))(0) + cachedTableCounts(i) = cachedCount + } + sharkMetastore.shutdown() + for ((tableName, i) <- globalCachedTableNames.zipWithIndex) { + val hiveTable = Hive.get().getTable(DEFAULT_DB_NAME, tableName) + // Check that the number of rows from the table on disk remains the same. + val onDiskCount = sc.sql("select count(*) from %s".format(tableName))(0) + val cachedCount = cachedTableCounts(i) + assert(onDiskCount == cachedCount, """Num rows for %s differ across Shark metastore restart. + (rows cached = %s, rows on disk = %s)""".format(tableName, cachedCount, onDiskCount)) + // Check that we're able to materialize a row - i.e., make sure that table scan operator + // doesn't try to use a ColumnarSerDe when scanning contents on disk (for our test tables, + // LazySimpleSerDes should be used). + sc.sql("select * from %s limit 1".format(tableName)) + } + // Finally, reload all tables. + SharkRunner.loadTables() + } + + ////////////////////////////////////////////////////////////////////////////// + // Window Function Support + ////////////////////////////////////////////////////////////////////////////// + test("window function support") { + expectSql("select id,name,count(id) over (partition by name) from users", + Array[String]("1\tA\t2", "3\tA\t2", "2\tB\t1")) + expectSql("select id,name,sum(id) over(partition by name order by id) from users", + Array[String]("1\tA\t1", "3\tA\t4", "2\tB\t2")) + expectSql("select id,name,sum(id) over(partition by name order by id rows between " + + "unbounded preceding and current row) from users", + Array[String]("1\tA\t1", "3\tA\t4", "2\tB\t2")) + expectSql("select id,name,sum(id) over(partition by name order by id rows between " + + "current row and unbounded following) from users", + Array[String]("1\tA\t4", "3\tA\t3", "2\tB\t2")) + expectSql("select id,name,sum(id) over(partition by name order by id rows between " + + "unbounded preceding and unbounded following) from users", + Array[String]("1\tA\t4", "3\tA\t4", "2\tB\t2")) + expectSql("select id,name,lead(id) over(partition by name order by id) from users", + Array[String]("1\tA\t3", "3\tA\tnull", "2\tB\tnull")) + expectSql("select id,name,lag(id) over(partition by name order by id) from users", + Array[String]("1\tA\tnull", "3\tA\t1", "2\tB\tnull")) + expectSql("select id, name, sum(id) over w1 as sum_id, max(id) over w1 as max_id from users" + + " window w1 as (partition by name)", + Array[String]("2\tB\t2\t2","1\tA\t4\t3","3\tA\t4\t3")) + } + + ////////////////////////////////////////////////////////////////////////////// + // Table Generating Functions (TGFs) + ////////////////////////////////////////////////////////////////////////////// + + test("Simple TGFs") { + expectSql("generate shark.TestTGF1(test, 15)", Array(15,15,15,17,19).map(_.toString).toArray) + } + + test("Saving simple TGFs") { + sc.sql("drop table if exists TGFTestTable") + sc.runSql("generate shark.TestTGF1(test, 15) as TGFTestTable") + expectSql("select * from TGFTestTable", Array(15,15,15,17,19).map(_.toString).toArray) + sc.sql("drop table if exists TGFTestTable") + } + + test("Advanced TGFs") { + expectSql("generate shark.TestTGF2(test, 25)", Array(25,25,25,27,29).map(_.toString).toArray) + } + + test("Saving advanced TGFs") { + sc.sql("drop table if exists TGFTestTable2") + sc.runSql("generate shark.TestTGF2(test, 25) as TGFTestTable2") + expectSql("select * from TGFTestTable2", Array(25,25,25,27,29).map(_.toString).toArray) + sc.sql("drop table if exists TGFTestTable2") + } +} + +object TestTGF1 { + @Schema(spec = "values int") + def apply(test: RDD[(Int, String)], integer: Int) = { + test.map{ case Tuple2(k, v) => Tuple1(k + integer) }.filter{ case Tuple1(v) => v < 20 } + } +} + +object TestTGF2 { + def apply(sc: SharkContext, test: RDD[(Int, String)], integer: Int) = { + val rdd = test.map{ case Tuple2(k, v) => Seq(k + integer) }.filter{ case Seq(v) => v < 30 } + RDDSchema(rdd.asInstanceOf[RDD[Seq[_]]], "myvalues int") + } +} diff --git a/src/main/test/scala/shark/SharkRunner.scala b/src/main/test/scala/shark/SharkRunner.scala new file mode 100755 index 00000000..01e69173 --- /dev/null +++ b/src/main/test/scala/shark/SharkRunner.scala @@ -0,0 +1,127 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark + +import org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME + +import shark.api.JavaSharkContext +import shark.memstore2.MemoryMetadataManager + + +object SharkRunner { + + val WAREHOUSE_PATH = TestUtils.getWarehousePath() + val METASTORE_PATH = TestUtils.getMetastorePath() + val MASTER = "local" + + var sc: SharkContext = _ + + var javaSc: JavaSharkContext = _ + + def init(): SharkContext = synchronized { + if (sc == null) { + sc = SharkEnv.initWithSharkContext("shark-sql-suite-testing", MASTER) + + sc.runSql("set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + + METASTORE_PATH + ";create=true") + sc.runSql("set hive.metastore.warehouse.dir=" + WAREHOUSE_PATH) + sc.runSql("set shark.test.data.path=" + TestUtils.dataFilePath) + + // second db + sc.sql("create database if not exists seconddb") + + loadTables() + } + sc + } + + def initWithJava(): JavaSharkContext = synchronized { + if (javaSc == null) { + javaSc = new JavaSharkContext(init()) + } + javaSc + } + + /** + * Tables accessible by any test. Their properties should remain constant across + * tests. + */ + def loadTables() = synchronized { + require(sc != null, "call init() to instantiate a SharkContext first") + + // Use the default namespace + sc.runSql("USE " + DEFAULT_DATABASE_NAME) + + // test + sc.runSql("drop table if exists test") + sc.runSql("CREATE TABLE test (key INT, val STRING)") + sc.runSql("LOAD DATA LOCAL INPATH '${hiveconf:shark.test.data.path}/kv1.txt' INTO TABLE test") + sc.runSql("drop table if exists test_cached") + sc.runSql("CREATE TABLE test_cached tblproperties('shark.cache'='memory') AS SELECT * FROM test") + + // test_null + sc.runSql("drop table if exists test_null") + sc.runSql("CREATE TABLE test_null (key INT, val STRING)") + sc.runSql("""LOAD DATA LOCAL INPATH '${hiveconf:shark.test.data.path}/kv3.txt' + INTO TABLE test_null""") + sc.runSql("drop table if exists test_null_cached") + sc.runSql("CREATE TABLE test_null_cached tblproperties('shark.cache'='memory') AS SELECT * FROM test_null") + + // clicks + sc.runSql("drop table if exists clicks") + sc.runSql("""create table clicks (id int, click int) + row format delimited fields terminated by '\t'""") + sc.runSql("""load data local inpath '${hiveconf:shark.test.data.path}/clicks.txt' + OVERWRITE INTO TABLE clicks""") + sc.runSql("drop table if exists clicks_cached") + sc.runSql("create table clicks_cached tblproperties('shark.cache'='memory') as select * from clicks") + + // users + sc.runSql("drop table if exists users") + sc.runSql("""create table users (id int, name string) + row format delimited fields terminated by '\t'""") + sc.runSql("""load data local inpath '${hiveconf:shark.test.data.path}/users.txt' + OVERWRITE INTO TABLE users""") + sc.runSql("drop table if exists users_cached") + sc.runSql("create table users_cached tblproperties('shark.cache'='memory') as select * from users") + + // test1 + sc.sql("drop table if exists test1") + sc.sql("""CREATE TABLE test1 (id INT, test1val ARRAY) + row format delimited fields terminated by '\t'""") + sc.sql("LOAD DATA LOCAL INPATH '${hiveconf:shark.test.data.path}/test1.txt' INTO TABLE test1") + sc.sql("drop table if exists test1_cached") + sc.sql("CREATE TABLE test1_cached tblproperties('shark.cache'='memory') AS SELECT * FROM test1") + Unit + } + + def expectSql(sql: String, expectedResults: Array[String], sort: Boolean = true) { + val sharkResults: Array[String] = sc.runSql(sql).results.map(_.mkString("\t")).toArray + val results = if (sort) sharkResults.sortWith(_ < _) else sharkResults + val expected = if (sort) expectedResults.sortWith(_ < _) else expectedResults + assert(results.corresponds(expected)(_.equals(_)), + "In SQL: " + sql + "\n" + + "Expected: " + expected.mkString("\n") + "; got " + results.mkString("\n")) + } + + // A shortcut for single row results. + def expectSql(sql: String, expectedResult: String) { + expectSql(sql, Array(expectedResult)) + } + +} diff --git a/src/main/test/scala/shark/SharkServerSuite.scala b/src/main/test/scala/shark/SharkServerSuite.scala new file mode 100755 index 00000000..686dc193 --- /dev/null +++ b/src/main/test/scala/shark/SharkServerSuite.scala @@ -0,0 +1,108 @@ +package shark + +import java.io.{BufferedReader, InputStreamReader} +import java.sql.DriverManager +import java.sql.Statement +import java.sql.Connection + +import scala.collection.JavaConversions._ + +import org.scalatest.{BeforeAndAfterAll, FunSuite} +import org.scalatest.matchers.ShouldMatchers + +import scala.concurrent._ +import ExecutionContext.Implicits.global + +/** + * Test for the Shark server. + */ +class SharkServerSuite extends FunSuite with BeforeAndAfterAll with ShouldMatchers with TestUtils { + + val WAREHOUSE_PATH = TestUtils.getWarehousePath("server") + val METASTORE_PATH = TestUtils.getMetastorePath("server") + val DRIVER_NAME = "org.apache.hadoop.hive.jdbc.HiveDriver" + val TABLE = "test" + // use a different port, than the hive standard 10000, + // for tests to avoid issues with the port being taken on some machines + val PORT = "9011" + + // If verbose is true, the testing program will print all outputs coming from the shark server. + val VERBOSE = Option(System.getenv("SHARK_TEST_VERBOSE")).getOrElse("false").toBoolean + + Class.forName(DRIVER_NAME) + + override def beforeAll() { launchServer() } + + override def afterAll() { stopServer() } + + private def launchServer(args: Seq[String] = Seq.empty) { + // Forking a new process to start the Shark server. The reason to do this is it is + // hard to clean up Hive resources entirely, so we just start a new process and kill + // that process for cleanup. + val defaultArgs = Seq("./bin/shark", "--service", "sharkserver", + "--verbose", + "-p", + PORT, + "--hiveconf", + "hive.root.logger=INFO,console", + "--hiveconf", + "\"javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + METASTORE_PATH + ";create=true\"", + "--hiveconf", + "\"hive.metastore.warehouse.dir=" + WAREHOUSE_PATH + "\"") + val pb = new ProcessBuilder(defaultArgs ++ args) + process = pb.start() + inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) + errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) + waitForOutput(inputReader, "Starting Shark server") + + // Spawn a thread to read the output from the forked process. + // Note that this is necessary since in some configurations, log4j could be blocked + // if its output to stderr are not read, and eventually blocking the entire test suite. + future { + while (true) { + val stdout = readFrom(inputReader) + val stderr = readFrom(errorReader) + if (VERBOSE && stdout.length > 0) { + println(stdout) + } + if (VERBOSE && stderr.length > 0) { + println(stderr) + } + Thread.sleep(50) + } + } + } + + private def stopServer() { + process.destroy() + process.waitFor() + } + + test("test query execution against a shark server") { + Thread.sleep(5*1000) // I know... Gross. However, without this the tests fail non-deterministically. + + val dataFilePath = TestUtils.dataFilePath + "/kv1.txt" + val stmt = createStatement() + stmt.executeQuery("DROP TABLE IF EXISTS test") + stmt.executeQuery("DROP TABLE IF EXISTS test_cached") + stmt.executeQuery("CREATE TABLE test(key int, val string)") + stmt.executeQuery("LOAD DATA LOCAL INPATH '" + dataFilePath+ "' OVERWRITE INTO TABLE test") + stmt.executeQuery("CREATE TABLE test_cached as select * from test limit 499") + + var rs = stmt.executeQuery("select count(*) from test") + rs.next() + rs.getInt(1) should equal (500) + + rs = stmt.executeQuery("select count(*) from test_cached") + rs.next() + rs.getInt(1) should equal (499) + + stmt.close() + } + + def getConnection(): Connection = { + DriverManager.getConnection("jdbc:hive://localhost:" + PORT + "/default", "", "") + } + + def createStatement(): Statement = getConnection().createStatement() +} \ No newline at end of file diff --git a/src/main/test/scala/shark/SortSuite.scala b/src/main/test/scala/shark/SortSuite.scala new file mode 100755 index 00000000..ead1e4b5 --- /dev/null +++ b/src/main/test/scala/shark/SortSuite.scala @@ -0,0 +1,53 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark + +import org.apache.hadoop.io.BytesWritable + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD + +import shark.execution.{ReduceKey, ReduceKeyMapSide, ReduceKeyReduceSide, RDDUtils} + + +class SortSuite extends FunSuite { + + TestUtils.init() + + var sc: SparkContext = SharkRunner.init() + + test("order by limit") { + val data = Array((4, 14), (1, 11), (7, 17), (0, 10)) + val expected = data.sortWith(_._1 < _._1).toSeq + val rdd: RDD[(ReduceKey, BytesWritable)] = sc.parallelize(data, 50).map { x => + (new ReduceKeyMapSide(new BytesWritable(Array[Byte](x._1.toByte))), + new BytesWritable(Array[Byte](x._2.toByte))) + } + for (k <- 0 to 5) { + val sortedRdd = RDDUtils.topK(rdd, k).asInstanceOf[RDD[(ReduceKeyReduceSide, Array[Byte])]] + val output = sortedRdd.map { case(k, v) => + (k.byteArray(0).toInt, v(0).toInt) + }.collect().toSeq + assert(output.size === math.min(k, 4)) + assert(output === expected.take(math.min(k, 4))) + } + } + +} diff --git a/src/main/test/scala/shark/TachyonSQLSuite.scala b/src/main/test/scala/shark/TachyonSQLSuite.scala new file mode 100755 index 00000000..c808f242 --- /dev/null +++ b/src/main/test/scala/shark/TachyonSQLSuite.scala @@ -0,0 +1,436 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark + +import java.util.{HashMap => JavaHashMap} + +import scala.collection.JavaConversions._ + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME +import org.apache.hadoop.hive.ql.metadata.Hive +import org.apache.spark.rdd.UnionRDD +import org.apache.spark.storage.StorageLevel + +import shark.api.QueryExecutionException +import shark.memstore2.{OffHeapStorageClient, CacheType, MemoryMetadataManager, PartitionedMemoryTable} +// import expectSql() shortcut methods +import shark.SharkRunner._ + + +class TachyonSQLSuite extends FunSuite with BeforeAndAfterAll { + + val DEFAULT_DB_NAME = DEFAULT_DATABASE_NAME + val KV1_TXT_PATH = "${hiveconf:shark.test.data.path}/kv1.txt" + + var sc: SharkContext = SharkRunner.init() + var sharkMetastore: MemoryMetadataManager = SharkEnv.memoryMetadataManager + + // Determine if Tachyon enabled at runtime. + val isTachyonEnabled = sys.env.contains("TACHYON_MASTER") + + override def beforeAll() { + if (isTachyonEnabled) { + sc.runSql("create table test_tachyon as select * from test") + } + } + + override def afterAll() { + if (isTachyonEnabled) { + sc.runSql("drop table test_tachyon") + } + } + + private def isTachyonTable( + dbName: String, + tableName: String, + hivePartitionKeyOpt: Option[String] = None): Boolean = { + val tableKey = MemoryMetadataManager.makeTableKey(dbName, tableName) + OffHeapStorageClient.client.tablePartitionExists(tableKey, hivePartitionKeyOpt) + } + + private def createPartitionedTachyonTable(tableName: String, numPartitionsToCreate: Int) { + sc.runSql("drop table if exists %s".format(tableName)) + sc.runSql(""" + create table %s(key int, value string) + partitioned by (keypart int) + tblproperties('shark.cache' = 'tachyon') + """.format(tableName)) + var partitionNum = 1 + while (partitionNum <= numPartitionsToCreate) { + sc.runSql("""insert into table %s partition(keypart = %d) + select * from test_tachyon""".format(tableName, partitionNum)) + partitionNum += 1 + } + assert(isTachyonTable(DEFAULT_DB_NAME, tableName)) + } + + if (isTachyonEnabled) { + ////////////////////////////////////////////////////////////////////////////// + // basic SQL + ////////////////////////////////////////////////////////////////////////////// + test("count") { + expectSql("select count(*) from test_tachyon", "500") + } + + test("filter") { + expectSql("select * from test_tachyon where key=100 or key=497", + Array("100\tval_100", "100\tval_100", "497\tval_497")) + } + + test("count distinct") { + sc.runSql("set mapred.reduce.tasks=3") + expectSql("select count(distinct key) from test_tachyon", "309") + expectSql( + """|SELECT substr(key,1,1), count(DISTINCT substr(val,5)) from test_tachyon + |GROUP BY substr(key,1,1)""".stripMargin, + Array("0\t1", "1\t71", "2\t69", "3\t62", "4\t74", "5\t6", "6\t5", "7\t6", "8\t8", "9\t7")) + } + + test("count bigint") { + sc.runSql("drop table if exists test_bigint") + sc.runSql("create table test_bigint (key bigint, val string)") + sc.runSql("""load data local inpath '${hiveconf:shark.test.data.path}/kv1.txt' + OVERWRITE INTO TABLE test_bigint""") + sc.runSql("drop table if exists test_bigint_tachyon") + sc.runSql("create table test_bigint_tachyon as select * from test_bigint") + expectSql("select val, count(*) from test_bigint_tachyon where key=484 group by val", + "val_484\t1") + + sc.runSql("drop table if exists test_bigint_tachyon") + } + + test("limit") { + assert(sc.runSql("select * from test_tachyon limit 10").results.length === 10) + assert(sc.runSql("select * from test_tachyon limit 501").results.length === 500) + sc.runSql("drop table if exists test_limit0_tachyon") + assert(sc.runSql("select * from test_tachyon limit 0").results.length === 0) + assert(sc.runSql("create table test_limit0_tachyon as select * from test_tachyon limit 0") + .results.length === 0) + assert(sc.runSql("select * from test_limit0_tachyon limit 0").results.length === 0) + assert(sc.runSql("select * from test_limit0_tachyon limit 1").results.length === 0) + + sc.runSql("drop table if exists test_limit0_tachyon") + } + + ////////////////////////////////////////////////////////////////////////////// + // cache DDL + ////////////////////////////////////////////////////////////////////////////// + test("Use regular CREATE TABLE and '_tachyon' suffix to create Tachyon table") { + sc.runSql("drop table if exists empty_table_tachyon") + sc.runSql("create table empty_table_tachyon(key string, value string)") + assert(isTachyonTable(DEFAULT_DB_NAME, "empty_table_tachyon")) + + sc.runSql("drop table if exists empty_table_tachyon") + } + + test("Use regular CREATE TABLE and table properties to create Tachyon table") { + sc.runSql("drop table if exists empty_table_tachyon_tbl_props") + sc.runSql("""create table empty_table_tachyon_tbl_props(key string, value string) + TBLPROPERTIES('shark.cache' = 'tachyon')""") + assert(isTachyonTable(DEFAULT_DB_NAME, "empty_table_tachyon_tbl_props")) + + sc.runSql("drop table if exists empty_table_tachyon_tbl_props") + } + + test("Insert into empty Tachyon table") { + sc.runSql("drop table if exists new_table_tachyon") + sc.runSql("create table new_table_tachyon(key string, value string)") + sc.runSql("insert into table new_table_tachyon select * from test where key > -1 limit 499") + expectSql("select count(*) from new_table_tachyon", "499") + + sc.runSql("drop table if exists new_table_tachyon") + } + + test("rename Tachyon table") { + sc.runSql("drop table if exists test_oldname_tachyon") + sc.runSql("drop table if exists test_rename") + sc.runSql("create table test_oldname_tachyon as select * from test") + sc.runSql("alter table test_oldname_tachyon rename to test_rename") + + assert(!isTachyonTable(DEFAULT_DB_NAME, "test_oldname_tachyon")) + assert(isTachyonTable(DEFAULT_DB_NAME, "test_rename")) + + expectSql("select count(*) from test_rename", "500") + + sc.runSql("drop table if exists test_rename") + } + + test("insert into tachyon tables") { + sc.runSql("drop table if exists test1_tachyon") + sc.runSql("create table test1_tachyon as select * from test") + expectSql("select count(*) from test1_tachyon", "500") + sc.runSql("insert into table test1_tachyon select * from test where key > -1 limit 499") + expectSql("select count(*) from test1_tachyon", "999") + + sc.runSql("drop table if exists test1_tachyon") + } + + test("insert overwrite") { + sc.runSql("drop table if exists test2_tachyon") + sc.runSql("create table test2_tachyon as select * from test") + expectSql("select count(*) from test2_tachyon", "500") + sc.runSql("insert overwrite table test2_tachyon select * from test where key > -1 limit 499") + expectSql("select count(*) from test2_tachyon", "499") + + sc.runSql("drop table if exists test2_tachyon") + } + + test("error when attempting to update Tachyon table(s) using command with multiple INSERTs") { + sc.runSql("drop table if exists multi_insert_test") + sc.runSql("drop table if exists multi_insert_test_tachyon") + sc.runSql("create table multi_insert_test as select * from test") + sc.runSql("create table multi_insert_test_tachyon as select * from test") + intercept[QueryExecutionException] { + sc.runSql("""from test + insert into table multi_insert_test select * + insert into table multi_insert_test_tachyon select *""") + } + + sc.runSql("drop table if exists multi_insert_test") + sc.runSql("drop table if exists multi_insert_test_tachyon") + } + + test("create Tachyon table with 'shark.cache' flag in table properties") { + sc.runSql("drop table if exists ctas_tbl_props") + sc.runSql("""create table ctas_tbl_props TBLPROPERTIES ('shark.cache'='tachyon') as + select * from test""") + assert(isTachyonTable(DEFAULT_DB_NAME, "ctas_tbl_props")) + expectSql("select * from ctas_tbl_props where key=407", "407\tval_407") + + sc.runSql("drop table if exists ctas_tbl_props") + } + + test("tachyon tables with complex types") { + sc.runSql("drop table if exists test_complex_types") + sc.runSql("drop table if exists test_complex_types_tachyon") + sc.runSql("""CREATE TABLE test_complex_types ( + a STRING, b ARRAY, c ARRAY>, d MAP>)""") + sc.runSql("""load data local inpath '${hiveconf:shark.test.data.path}/create_nested_type.txt' + overwrite into table test_complex_types""") + sc.runSql("""create table test_complex_types_tachyon TBLPROPERTIES ("shark.cache" = "tachyon") + as select * from test_complex_types""") + + assert(sc.sql("select a from test_complex_types_tachyon where a = 'a0'").head === "a0") + + assert(sc.sql("select b from test_complex_types_tachyon where a = 'a0'").head === + """["b00","b01"]""") + + assert(sc.sql("select c from test_complex_types_tachyon where a = 'a0'").head === + """[{"c001":"C001","c002":"C002"},{"c011":null,"c012":"C012"}]""") + + assert(sc.sql("select d from test_complex_types_tachyon where a = 'a0'").head === + """{"d01":["d011","d012"],"d02":["d021","d022"]}""") + + assert(isTachyonTable(DEFAULT_DB_NAME, "test_complex_types_tachyon")) + + sc.runSql("drop table if exists test_complex_types") + sc.runSql("drop table if exists test_complex_types_tachyon") + } + + test("disable caching in Tachyon by default") { + sc.runSql("set shark.cache.flag.checkTableName=false") + sc.runSql("drop table if exists should_not_be_in_tachyon") + sc.runSql("create table should_not_be_in_tachyon as select * from test") + expectSql("select key from should_not_be_in_tachyon where key = 407", "407") + assert(!isTachyonTable(DEFAULT_DB_NAME, "should_not_be_in_tachyon")) + + sc.runSql("set shark.cache.flag.checkTableName=true") + sc.runSql("drop table if exists should_not_be_in_tachyon") + } + + test("tachyon table name should be case-insensitive") { + sc.runSql("drop table if exists sharkTest5tachyon") + sc.runSql("""create table sharkTest5tachyon TBLPROPERTIES ("shark.cache" = "tachyon") as + select * from test""") + expectSql("select val from sharktest5tachyon where key = 407", "val_407") + assert(isTachyonTable(DEFAULT_DB_NAME, "sharkTest5tachyon")) + + sc.runSql("drop table if exists sharkTest5tachyon") + } + + test("dropping tachyon tables should clean up RDDs") { + sc.runSql("drop table if exists sharkTest5tachyon") + sc.runSql("""create table sharkTest5tachyon TBLPROPERTIES ("shark.cache" = "tachyon") as + select * from test""") + sc.runSql("drop table sharkTest5tachyon") + assert(!isTachyonTable(DEFAULT_DB_NAME, "sharkTest5tachyon")) + } + + ////////////////////////////////////////////////////////////////////////////// + // Caching Hive-partititioned tables + // Note: references to 'partition' for this section refer to a Hive-partition. + ////////////////////////////////////////////////////////////////////////////// + test("Use regular CREATE TABLE and '_tachyon' suffix to create partitioned Tachyon table") { + sc.runSql("drop table if exists empty_part_table_tachyon") + sc.runSql("""create table empty_part_table_tachyon(key int, value string) + partitioned by (keypart int)""") + assert(isTachyonTable(DEFAULT_DB_NAME, "empty_part_table_tachyon")) + + sc.runSql("drop table if exists empty_part_table_tachyon") + } + + test("Use regular CREATE TABLE and table properties to create partitioned Tachyon table") { + sc.runSql("drop table if exists empty_part_table_tachyon_tbl_props") + sc.runSql("""create table empty_part_table_tachyon_tbl_props(key int, value string) + partitioned by (keypart int) tblproperties('shark.cache' = 'tachyon')""") + assert(isTachyonTable(DEFAULT_DB_NAME, "empty_part_table_tachyon_tbl_props")) + + sc.runSql("drop table if exists empty_part_table_tachyon_tbl_props") + } + + test("alter Tachyon table by adding a new partition") { + sc.runSql("drop table if exists alter_part_tachyon") + sc.runSql("""create table alter_part_tachyon(key int, value string) + partitioned by (keypart int)""") + sc.runSql("""alter table alter_part_tachyon add partition(keypart = 1)""") + val tableName = "alter_part_tachyon" + val partitionColumn = "keypart=1" + assert(isTachyonTable(DEFAULT_DB_NAME, "alter_part_tachyon", Some(partitionColumn))) + + sc.runSql("drop table if exists alter_part_tachyon") + } + + test("alter Tachyon table by dropping a partition") { + sc.runSql("drop table if exists alter_drop_tachyon") + sc.runSql("""create table alter_drop_tachyon(key int, value string) + partitioned by (keypart int)""") + sc.runSql("""alter table alter_drop_tachyon add partition(keypart = 1)""") + + val tableName = "alter_drop_tachyon" + val partitionColumn = "keypart=1" + assert(isTachyonTable(DEFAULT_DB_NAME, "alter_drop_tachyon", Some(partitionColumn))) + sc.runSql("""alter table alter_drop_tachyon drop partition(keypart = 1)""") + assert(!isTachyonTable(DEFAULT_DB_NAME, "alter_drop_tachyon", Some(partitionColumn))) + + sc.runSql("drop table if exists alter_drop_tachyon") + } + + test("insert into a partition of a Tachyon table") { + val tableName = "insert_part_tachyon" + createPartitionedTachyonTable( + tableName, + numPartitionsToCreate = 1) + expectSql("select value from insert_part_tachyon where key = 407 and keypart = 1", "val_407") + + sc.runSql("drop table if exists insert_part_tachyon") + } + + test("insert overwrite a partition of a Tachyon table") { + val tableName = "insert_over_part_tachyon" + createPartitionedTachyonTable( + tableName, + numPartitionsToCreate = 1) + expectSql("""select value from insert_over_part_tachyon + where key = 407 and keypart = 1""", "val_407") + sc.runSql("""insert overwrite table insert_over_part_tachyon partition(keypart = 1) + select key, -1 from test""") + expectSql("select value from insert_over_part_tachyon where key = 407 and keypart = 1", "-1") + + sc.runSql("drop table if exists insert_over_part_tachyon") + } + + test("scan partitioned Tachyon table that's empty") { + sc.runSql("drop table if exists empty_part_table_tachyon") + sc.runSql("""create table empty_part_table_tachyon(key int, value string) + partitioned by (keypart int)""") + expectSql("select count(*) from empty_part_table_tachyon", "0") + + sc.runSql("drop table if exists empty_part_table_tachyon") + } + + test("scan partitioned Tachyon table that has a single partition") { + val tableName = "scan_single_part_tachyon" + createPartitionedTachyonTable( + tableName, + numPartitionsToCreate = 1) + expectSql("select * from scan_single_part_tachyon where key = 407", "407\tval_407\t1") + + sc.runSql("drop table if exists scan_single_part_tachyon") + } + + test("scan partitioned Tachyon table that has multiple partitions") { + val tableName = "scan_mult_part_tachyon" + createPartitionedTachyonTable( + tableName, + numPartitionsToCreate = 3) + expectSql("select * from scan_mult_part_tachyon where key = 407 order by keypart", + Array("407\tval_407\t1", "407\tval_407\t2", "407\tval_407\t3")) + + sc.runSql("drop table if exists scan_mult_part_tachyon") + } + + test("drop/unpersist partitioned Tachyon table that has multiple partitions") { + val tableName = "drop_mult_part_tachyon" + createPartitionedTachyonTable( + tableName, + numPartitionsToCreate = 3) + expectSql("select count(1) from drop_mult_part_tachyon", "1500") + sc.runSql("drop table drop_mult_part_tachyon ") + assert(!isTachyonTable(DEFAULT_DB_NAME, tableName)) + + sc.runSql("drop table if exists drop_mult_part_tachyon") + } + + ///////////////////////////////////////////////////////////////////////////// + // LOAD for Tachyon tables + ////////////////////////////////////////////////////////////////////////////// + test ("LOAD INTO a Tachyon table") { + sc.runSql("drop table if exists load_into_tachyon") + sc.runSql("create table load_into_tachyon (key int, value string)") + sc.runSql("load data local inpath '%s' into table load_into_tachyon".format(KV1_TXT_PATH)) + expectSql("select count(*) from load_into_tachyon", "500") + + sc.runSql("drop table if exists load_into_tachyon") + } + + test ("LOAD OVERWRITE a Tachyon table") { + sc.runSql("drop table if exists load_overwrite_tachyon") + sc.runSql("create table load_overwrite_tachyon (key int, value string)") + sc.runSql("load data local inpath '%s' into table load_overwrite_tachyon". + format("${hiveconf:shark.test.data.path}/kv3.txt")) + expectSql("select count(*) from load_overwrite_tachyon", "25") + sc.runSql("load data local inpath '%s' overwrite into table load_overwrite_tachyon". + format(KV1_TXT_PATH)) + expectSql("select count(*) from load_overwrite_tachyon", "500") + sc.runSql("drop table if exists load_overwrite_tachyon") + } + + test ("LOAD INTO a partitioned Tachyon table") { + sc.runSql("drop table if exists load_into_part_tachyon") + sc.runSql("""create table load_into_part_tachyon (key int, value string) + partitioned by (keypart int)""") + sc.runSql("""load data local inpath '%s' into table load_into_part_tachyon + partition(keypart = 1)""".format(KV1_TXT_PATH)) + expectSql("select count(*) from load_into_part_tachyon", "500") + sc.runSql("drop table if exists load_into_part_tachyon") + } + + test ("LOAD OVERWRITE a partitioned Tachyon table") { + sc.runSql("drop table if exists load_overwrite_part_tachyon") + sc.runSql("""create table load_overwrite_part_tachyon (key int, value string) + partitioned by (keypart int)""") + sc.runSql("""load data local inpath '%s' overwrite into table load_overwrite_part_tachyon + partition(keypart = 1)""".format(KV1_TXT_PATH)) + expectSql("select count(*) from load_overwrite_part_tachyon", "500") + sc.runSql("drop table if exists load_overwrite_part_tachyon") + } + } +} diff --git a/src/main/test/scala/shark/TestUtils.scala b/src/main/test/scala/shark/TestUtils.scala new file mode 100755 index 00000000..25d9b4af --- /dev/null +++ b/src/main/test/scala/shark/TestUtils.scala @@ -0,0 +1,195 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark + +import java.io.{BufferedReader, InputStreamReader, PrintWriter} +import java.text.SimpleDateFormat +import java.util.{Date, HashMap => JHashMap} + +import org.apache.hadoop.hive.common.LogUtils +import org.apache.hadoop.hive.common.LogUtils.LogInitializationException + +import org.apache.spark.rdd.{RDD, UnionRDD} +import org.apache.spark.storage.StorageLevel + + +object TestUtils { + + val timestamp = new SimpleDateFormat("yyyyMMdd-HHmmss") + + def getWarehousePath(prefix: String): String = { + System.getProperty("user.dir") + "/test_warehouses/" + prefix + "-warehouse-" + + timestamp.format(new Date) + } + + def getMetastorePath(prefix: String): String = { + System.getProperty("user.dir") + "/test_warehouses/" + prefix + "-metastore-" + + timestamp.format(new Date) + } + + def testAndSet(prefix: String): Boolean = synchronized { + if (testAndTestMap.get(prefix) == null) { + testAndTestMap.put(prefix, new Object) + true + } else { + false + } + } + + def getStorageLevelOfRDD(rdd: RDD[_]): StorageLevel = { + rdd match { + case u: UnionRDD[_] => { + // Find the storage level of a UnionRDD from the storage levels of RDDs that compose it. + // A StorageLevel.NONE is returned if all of those RDDs have StorageLevel.NONE. + // Mutually recursive if any RDD in 'u.rdds' is a UnionRDD. + getStorageLevelOfRDDs(u.rdds) + } + case _ => rdd.getStorageLevel + } + } + + /** + * Returns the storage level of a sequence of RDDs, interpreted as the storage level of the first + * RDD in the sequence that is persisted in memory or on disk. This works because for Shark's use + * case, all RDDs for a non-partitioned table should have the same storage level. An RDD for a + * partitioned table could be StorageLevel.NONE if it was unpersisted by the partition eviction + * policy. + * + * @param rdds The sequence of RDDs to find the StorageLevel of. + */ + def getStorageLevelOfRDDs(rdds: Seq[RDD[_]]): StorageLevel = { + rdds.foldLeft(StorageLevel.NONE) { + (s, r) => { + if (s == StorageLevel.NONE) { + // Mutally recursive if `r` is a UnionRDD. However, this shouldn't happen in Shark, since + // UnionRDDs from successive INSERT INTOs are created through #unionAndFlatten(). + getStorageLevelOfRDD(r) + } else { + // Some RDD in 'rdds' is persisted in memory or disk, so return early. + return s + } + } + } + } + + // Don't use default arguments in the above functions because otherwise the JavaAPISuite + // can't call those functions with default arguments. + def getWarehousePath(): String = getWarehousePath("sql") + def getMetastorePath(): String = getMetastorePath("sql") + def testAndSet(): Boolean = testAndSet("sql") + + private val testAndTestMap = new JHashMap[String, Object] + + def dataFilePath: String = { + Option(System.getenv("SHARK_HOME")).getOrElse(System.getProperty("user.dir")) + "/data/files" + } + + // Dummy function for initialize the log4j properties. + def init() { } + + // initialize log4j + try { + LogUtils.initHiveLog4j() + } catch { + case e: LogInitializationException => // Ignore the error. + } +} + + +trait TestUtils { + + var process : Process = null + var outputWriter : PrintWriter = null + var inputReader : BufferedReader = null + var errorReader : BufferedReader = null + + def dropTable(tableName: String, timeout: Long = 15000): String = { + executeQuery("drop table if exists " + tableName + ";") + } + + def executeQuery( + cmd: String, outputMessage: String = "OK", timeout: Long = 15000): String = { + println("Executing: " + cmd + ", expecting output: " + outputMessage) + outputWriter.write(cmd + "\n") + outputWriter.flush() + waitForQuery(timeout, outputMessage) + } + + protected def waitForQuery(timeout: Long, message: String): String = { + if (waitForOutput(errorReader, message, timeout)) { + Thread.sleep(500) + readOutput() + } else { + assert(false, "Didn't find \"" + message + "\" in the output:\n" + readOutput()) + null + } + } + + protected def waitForQuery(timeout: Long, message1: String, message2: String): String = { + if (waitForOutput2(errorReader, message1, message2, timeout)) { + Thread.sleep(500) + readOutput() + } else { + assert(false, "Didn't find '" + message1 + "' or '" + message2 + + "' in the output:\n" + readOutput()) + null + } + } + + // Wait for the specified str to appear in the output. + protected def waitForOutput( + reader: BufferedReader, str: String, timeout: Long = 10000): Boolean = { + val startTime = System.currentTimeMillis + var out = "" + while (!out.contains(str) && (System.currentTimeMillis) < (startTime + timeout)) { + out += readFrom(reader) + } + out.contains(str) + } + + // Wait for the specified str1 and str2 to appear in the output. + protected def waitForOutput2( + reader: BufferedReader, str1: String, str2: String, timeout: Long = 10000): Boolean = { + val startTime = System.currentTimeMillis + var out = "" + while (!out.contains(str1) && !out.contains(str2) && + (System.currentTimeMillis) < (startTime + timeout)) { + out += readFrom(reader) + } + out.contains(str1) || out.contains(str2) + } + + // Read stdout output from Shark and filter out garbage collection messages. + protected def readOutput(): String = { + val output = readFrom(inputReader) + // Remove GC Messages + val filteredOutput = output.lines.filterNot(x => x.contains("[GC") || x.contains("[Full GC")) + .mkString("\n") + filteredOutput + } + + protected def readFrom(reader: BufferedReader): String = { + var out = "" + var c = 0 + while (reader.ready) { + c = reader.read() + out += c.asInstanceOf[Char] + } + out + } +} diff --git a/src/main/test/scala/shark/UtilsSuite.scala b/src/main/test/scala/shark/UtilsSuite.scala new file mode 100755 index 00000000..7cb48a18 --- /dev/null +++ b/src/main/test/scala/shark/UtilsSuite.scala @@ -0,0 +1,55 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark + +import java.util.{HashMap => JHashMap} + +import org.apache.hadoop.conf.Configuration + +import org.scalatest.{BeforeAndAfter, FunSuite} + + +class UtilsSuite extends FunSuite { + + test("set aws credentials") { + var conf = new Configuration + var map = new JHashMap[String, String]() + Utils.setAwsCredentials(conf, map) + assert(conf.get("fs.s3n.awsAccessKeyId") === null) + assert(conf.get("fs.s3n.awsSecretAccessKey") === null) + assert(conf.get("fs.s3.awsAccessKeyId") === null) + assert(conf.get("fs.s3.awsSecretAccessKey") === null) + + map.put("AWS_ACCESS_KEY_ID", "id") + conf = new Configuration + Utils.setAwsCredentials(conf, map) + assert(conf.get("fs.s3n.awsAccessKeyId") === null) + assert(conf.get("fs.s3n.awsSecretAccessKey") === null) + assert(conf.get("fs.s3.awsAccessKeyId") === null) + assert(conf.get("fs.s3.awsSecretAccessKey") === null) + + map.put("AWS_SECRET_ACCESS_KEY", "key") + conf = new Configuration + Utils.setAwsCredentials(conf, map) + assert(conf.get("fs.s3n.awsAccessKeyId") === "id") + assert(conf.get("fs.s3n.awsSecretAccessKey") === "key") + assert(conf.get("fs.s3.awsAccessKeyId") === "id") + assert(conf.get("fs.s3.awsSecretAccessKey") === "key") + } + +} diff --git a/src/main/test/scala/shark/execution/HiveStructSerializerSuite.scala b/src/main/test/scala/shark/execution/HiveStructSerializerSuite.scala new file mode 100755 index 00000000..ac646954 --- /dev/null +++ b/src/main/test/scala/shark/execution/HiveStructSerializerSuite.scala @@ -0,0 +1,65 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.execution + +import java.util.{ArrayList => JArrayList} + +import scala.collection.JavaConversions._ + +import org.apache.hadoop.hive.serde2.binarysortable.{HiveStructSerializer, HiveStructDeserializer} +import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, + ObjectInspectorFactory, StandardListObjectInspector, StandardMapObjectInspector, + StructObjectInspector} +import org.apache.hadoop.hive.serde2.objectinspector.primitive.{PrimitiveObjectInspectorUtils, + PrimitiveObjectInspectorFactory} +import org.apache.hadoop.io.{IntWritable, LongWritable, Text} + +import org.scalatest.FunSuite + + +class HiveStructSerializerSuite extends FunSuite { + + test("Testing serializing a simple row") { + val row1 = createRow(1, "test1") + val row2 = createRow(2, "test2") + val ser = new HiveStructSerializer(createObjectInspector) + val deser = new HiveStructDeserializer(createObjectInspector) + val deserRow1 = deser.deserialize(ser.serialize(row1)) + assert(row1.get(0).equals(deserRow1.get(0))) + assert(row1.get(1).equals(deserRow1.get(1))) + } + + def createObjectInspector(): StructObjectInspector = { + val names = List("a", "b") + val ois = List( + createPrimitiveOi(classOf[java.lang.Integer]), + createPrimitiveOi(classOf[String])) + ObjectInspectorFactory.getStandardStructObjectInspector(names, ois) + } + + def createRow(v1: Int, v2: String): JArrayList[Object] = { + val row = new JArrayList[Object](2) + row.add(new IntWritable(v1)) + row.add(new Text(v2)) + row + } + + def createPrimitiveOi(javaClass: Class[_]): PrimitiveObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector( + PrimitiveObjectInspectorUtils.getTypeEntryFromPrimitiveJavaClass(javaClass).primitiveCategory) +} diff --git a/src/main/test/scala/shark/execution/serialization/SerializationSuite.scala b/src/main/test/scala/shark/execution/serialization/SerializationSuite.scala new file mode 100755 index 00000000..297b7379 --- /dev/null +++ b/src/main/test/scala/shark/execution/serialization/SerializationSuite.scala @@ -0,0 +1,109 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.execution.serialization + +import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer + +import org.apache.commons.lang.ObjectUtils +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory +import org.apache.hadoop.hive.conf.HiveConf + +import org.scalatest.FunSuite + +import org.apache.spark.SparkConf +import org.apache.spark.serializer.{JavaSerializer => SparkJavaSerializer} + + +object SerializationSuite { + val DEPRECATED_HIVECONF_PROPERTIES = List( + "topology.script.file.name", + "topology.script.number.args", + "hadoop.configured.node.mapping", + "topology.node.switch.mapping.impl", + "dfs.df.interval", + "dfs.client.buffer.dir", + "hadoop.native.lib", + "fs.default.name" + ).toSet[String] +} + +class SerializationSuite extends FunSuite { + + test("Java serializing object inspectors") { + + val oi = PrimitiveObjectInspectorFactory.javaStringObjectInspector + val ois = KryoSerializationWrapper(new ArrayBuffer[ObjectInspector]) + ois.value += oi + + val ser = new SparkJavaSerializer(new SparkConf(loadDefaults = false)) + val bytes = ser.newInstance().serialize(ois) + val desered = ser.newInstance() + .deserialize[KryoSerializationWrapper[ArrayBuffer[ObjectInspector]]](bytes) + + assert(desered.head.getTypeName() === oi.getTypeName()) + } + + test("Java serializing operators") { + import shark.execution.{FileSinkOperator => SharkFileSinkOperator} + + val operator = new SharkFileSinkOperator + operator.localHconf = new org.apache.hadoop.hive.conf.HiveConf + operator.localHiveOp = new org.apache.hadoop.hive.ql.exec.FileSinkOperator + val opWrapped = OperatorSerializationWrapper(operator) + + val ser = new SparkJavaSerializer(new SparkConf(loadDefaults = false)) + val bytes = ser.newInstance().serialize(opWrapped) + val desered = ser.newInstance() + .deserialize[OperatorSerializationWrapper[SharkFileSinkOperator]](bytes) + + assert(desered.value != null) + assert(desered.value.localHconf != null) + assert(desered.value.localHiveOp != null) + assertHiveConfEquals(opWrapped.localHconf, desered.value.localHconf) + } + + test("XMLEncoder HiveConfPersistenceDelegate") { + val hiveConf = new HiveConf + hiveConf.setAuxJars("hive-aux.jar") + hiveConf.set("test.test.test", "true") + val bytes = XmlSerializer.serialize(hiveConf, hiveConf) + val deseredConf = XmlSerializer.deserialize[org.apache.hadoop.hive.conf.HiveConf](bytes) + + assertHiveConfEquals(hiveConf, deseredConf) + } + + def assertHiveConfEquals(expectedConf: HiveConf, resultConf: HiveConf) = { + expectedConf.getAllProperties.entrySet().foreach { entry => + if (!SerializationSuite.DEPRECATED_HIVECONF_PROPERTIES.contains(entry.getKey)) { + assert(resultConf.getAllProperties.get(entry.getKey()) === entry.getValue) + } + } + + resultConf.getAllProperties.entrySet().foreach { entry => + if (!SerializationSuite.DEPRECATED_HIVECONF_PROPERTIES.contains(entry.getKey)) { + assert(expectedConf.getAllProperties.get(entry.getKey()) === entry.getValue) + } + } + assert(ObjectUtils.equals(expectedConf.getAuxJars, resultConf.getAuxJars)) + } + +} + + diff --git a/src/main/test/scala/shark/execution/serialization/ShuffleSerializerSuite.scala b/src/main/test/scala/shark/execution/serialization/ShuffleSerializerSuite.scala new file mode 100755 index 00000000..aafa3d12 --- /dev/null +++ b/src/main/test/scala/shark/execution/serialization/ShuffleSerializerSuite.scala @@ -0,0 +1,112 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.execution.serialization + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream} + +import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} + +import org.apache.hadoop.io.BytesWritable +import org.scalatest.FunSuite +import org.scalatest.matchers.ShouldMatchers + +import shark.execution.{ReduceKey, ReduceKeyMapSide, ReduceKeyReduceSide} + + +class ShuffleSerializerSuite extends FunSuite with ShouldMatchers { + test("Encoding and decoding variable ints") { + val check = List[Int](0, 1000, 10000, 100000, 1000000, 10000000, 100000000, 1000000000) + + val bos = new ByteArrayOutputStream() + val ser = new ShuffleSerializer() + val serOutStream = { + ser.newInstance().serializeStream(bos).asInstanceOf[ShuffleSerializationStream] + } + for (i <- check) { + serOutStream.writeUnsignedVarInt(i) + } + serOutStream.close(); + + val bis = new ByteArrayInputStream(bos.toByteArray) + val serInStream = { + ser.newInstance().deserializeStream(bis).asInstanceOf[ShuffleDeserializationStream] + } + for (in <- check) { + val out: Int = serInStream.readUnsignedVarInt() + assert(out == in, "Encoded: " + in + " did not match decoded: " + out) + } + } + + test("Serializing and deserializing from a stream") { + val NUM_ITEMS = 5000 + val KEY_SIZE = 1000 + val VALUE_SIZE = 1000 + + val initialItems: Array[(ReduceKey, BytesWritable)] = Array.fill(NUM_ITEMS) { + val rkBytes = (1 to KEY_SIZE).map(_.toByte).toArray + val valueBytes = (1 to VALUE_SIZE).map(_.toByte).toArray + val rk = new ReduceKeyMapSide(new BytesWritable(rkBytes)) + val value = new BytesWritable(valueBytes) + (rk, value) + } + + val bos = new ByteArrayOutputStream() + val ser = new ShuffleSerializer() + val serStream = ser.newInstance().serializeStream(bos) + initialItems.map(serStream.writeObject(_)) + val bis = new ByteArrayInputStream(bos.toByteArray) + val serInStream = ser.newInstance().deserializeStream(bis) + + initialItems.foreach { expected: (ReduceKey, BytesWritable) => + val output: (ReduceKey, Array[Byte]) = serInStream.readObject() + (expected._1) should equal (output._1) + (expected._2.getBytes) should equal (output._2) + } + } + + test("Serializing and deserializing from a stream (with compression)") { + val NUM_ITEMS = 1000 + val KEY_SIZE = 1000 + val VALUE_SIZE = 1000 + + val initialItems = Array.fill(NUM_ITEMS) { + val rkBytes = (1 to KEY_SIZE).map(_.toByte).toArray + val valueBytes = (1 to VALUE_SIZE).map(_.toByte).toArray + val rk = new ReduceKeyMapSide(new BytesWritable(rkBytes)) + val value = new BytesWritable(valueBytes) + (rk, value) + } + + val bos = new ByteArrayOutputStream() + val cBos = new LZFOutputStream(bos).setFinishBlockOnFlush(true) + val ser = new ShuffleSerializer() + val serStream = ser.newInstance().serializeStream(cBos) + initialItems.map(serStream.writeObject(_)) + serStream.close() + val array = bos.toByteArray + val bis = new ByteArrayInputStream(array) + val cBis = new LZFInputStream(bis) + val serInStream = ser.newInstance().deserializeStream(cBis) + + initialItems.foreach { expected: (ReduceKey, BytesWritable) => + val output: (ReduceKey, Array[Byte]) = serInStream.readObject() + (expected._1) should equal (output._1) + (expected._2.getBytes) should equal (output._2) + } + } +} diff --git a/src/main/test/scala/shark/memstore2/CachePolicySuite.scala b/src/main/test/scala/shark/memstore2/CachePolicySuite.scala new file mode 100755 index 00000000..6eed3518 --- /dev/null +++ b/src/main/test/scala/shark/memstore2/CachePolicySuite.scala @@ -0,0 +1,131 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2 + +import org.scalatest.FunSuite + +import scala.collection.mutable.HashMap + +class CachePolicySuite extends FunSuite { + + case class TestValue(var value: Int, var isCached: Boolean) + + class IdentifyKVGen(max: Int) { + val kvMap = new HashMap[Int, TestValue]() + for (i <- 0 until max) { + kvMap(i) = TestValue(i, isCached = false) + } + + def loadFunc(key: Int) = { + val value = kvMap(key) + value.isCached = true + value + } + + def evictionFunc(key: Int, value: TestValue) = { + value.isCached = false + } + } + + test("LRU policy") { + val kvGen = new IdentifyKVGen(20) + val cacheSize = 10 + val lru = new LRUCachePolicy[Int, TestValue]() + lru.initialize(Array.empty[String], cacheSize, kvGen.loadFunc _, kvGen.evictionFunc _) + + // Load KVs 0-9. + (0 to 9).map(lru.notifyGet(_)) + assert(lru.keysOfCachedEntries.equals(Seq(0, 1, 2, 3, 4, 5, 6, 7, 8, 9))) + + // Reorder access order by getting keys 2-4. + (2 to 4).map(lru.notifyGet(_)) + assert(lru.keysOfCachedEntries.equals(Seq(0, 1, 5, 6, 7, 8, 9, 2, 3, 4))) + + // Get keys 10-12, which should evict (0, 1, 5). + (10 to 12).map(lru.notifyGet(_)) + assert(lru.keysOfCachedEntries.equals(Seq(6, 7, 8, 9, 2, 3, 4, 10, 11, 12))) + // Make sure the eviction function ran. + assert(!kvGen.kvMap(0).isCached) + assert(!kvGen.kvMap(1).isCached) + assert(!kvGen.kvMap(5).isCached) + + // Reorder access order by getting keys (6, 8, 2). + lru.notifyGet(6); lru.notifyGet(8); lru.notifyGet(2) + assert(lru.keysOfCachedEntries.equals(Seq(7, 9, 3, 4, 10, 11, 12, 6, 8, 2))) + + // Remove 9, 4 and add 13, 14, 15. 7 should be evicted. + lru.notifyRemove(9); lru.notifyRemove(4) + (13 to 15).map(lru.notifyGet(_)) + assert(lru.keysOfCachedEntries.equals(Seq(3, 10, 11, 12, 6, 8, 2, 13, 14, 15))) + assert(!kvGen.kvMap(7).isCached) + } + + test("FIFO policy") { + val kvGen = new IdentifyKVGen(15) + val cacheSize = 5 + val fifo = new FIFOCachePolicy[Int, TestValue]() + fifo.initialize(Array.empty[String], cacheSize, kvGen.loadFunc _, kvGen.evictionFunc _) + + // Load KVs 0-4. + (0 to 4).map(fifo.notifyPut(_, TestValue(-1, true))) + assert(fifo.keysOfCachedEntries.equals(Seq(0, 1, 2, 3, 4))) + + // Get 0-8, which should evict 0-3. + (0 to 8).map(fifo.notifyPut(_, TestValue(-1, true))) + assert(fifo.keysOfCachedEntries.equals(Seq(4, 5, 6, 7, 8))) + + // Remove 4, 6 and add 9-12. 5 and 7 should be evicted. + fifo.notifyRemove(4); fifo.notifyRemove(6) + (9 to 12).map(fifo.notifyPut(_, TestValue(-1, true))) + assert(fifo.keysOfCachedEntries.equals(Seq(8, 9, 10, 11, 12))) + } + + test("Policy classes instantiated from a string, with maxSize argument") { + val kvGen = new IdentifyKVGen(15) + val lruStr = "shark.memstore2.LRUCachePolicy(5)" + val lru = CachePolicy.instantiateWithUserSpecs( + lruStr, fallbackMaxSize = 10, kvGen.loadFunc _, kvGen.evictionFunc _) + assert(lru.maxSize == 5) + val fifoStr = "shark.memstore2.FIFOCachePolicy(5)" + val fifo = CachePolicy.instantiateWithUserSpecs( + fifoStr, fallbackMaxSize = 10, kvGen.loadFunc _, kvGen.evictionFunc _) + assert(fifo.maxSize == 5) + } + + test("Cache stats are recorded") { + val kvGen = new IdentifyKVGen(20) + val cacheSize = 5 + val lru = new LRUCachePolicy[Int, TestValue]() + lru.initialize(Array.empty[String], cacheSize, kvGen.loadFunc _, kvGen.evictionFunc _) + + // Hit rate should start at 1.0 + assert(lru.hitRate == 1.0) + + (0 to 4).map(lru.notifyGet(_)) + assert(lru.hitRate == 0.0) + + // Get 1, 2, 3, which should bring the hit rate to 0.375. + (1 to 3).map(lru.notifyGet(_)) + assert(lru.hitRate == 0.375) + + // Get 2-5, which brings the hit rate up to 0.50. + (2 to 5).map(lru.notifyGet(_)) + assert(lru.evictionCount == 1) + assert(lru.hitRate == 0.50) + } +} \ No newline at end of file diff --git a/src/main/test/scala/shark/memstore2/ColumnIteratorSuite.scala b/src/main/test/scala/shark/memstore2/ColumnIteratorSuite.scala new file mode 100755 index 00000000..8c297cac --- /dev/null +++ b/src/main/test/scala/shark/memstore2/ColumnIteratorSuite.scala @@ -0,0 +1,413 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2 + +import org.apache.hadoop.hive.serde2.`lazy`.ByteArrayRef +import org.apache.hadoop.hive.serde2.objectinspector.primitive._ +import org.apache.hadoop.io._ +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector + +import org.scalatest.FunSuite + +import shark.memstore2.column._ +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector +import shark.memstore2.column.Implicits._ +import java.nio.ByteOrder + + +class ColumnIteratorSuite extends FunSuite { + + val PARALLEL_MODE = true + + test("void column") { + val builder = new VoidColumnBuilder + builder.initialize(5, "void") + builder.append(null, null) + builder.append(null, null) + builder.append(null, null) + val buf = builder.build() + + val iter = ColumnIterator.newIterator(buf) + + iter.next() + + assert(iter.current == null) + iter.next() + assert(iter.current == null) + iter.next() + assert(iter.current == null) + } + + test("boolean column") { + var builder = new BooleanColumnBuilder + testColumn( + Array[java.lang.Boolean](true, false, true, true, true), + builder, + PrimitiveObjectInspectorFactory.javaBooleanObjectInspector, + PrimitiveObjectInspectorFactory.writableBooleanObjectInspector, + classOf[BooleanColumnIterator]) + assert(builder.stats.min === false) + assert(builder.stats.max === true) + + builder = new BooleanColumnBuilder + testColumn( + Array[java.lang.Boolean](null, false, null, true, true), + builder, + PrimitiveObjectInspectorFactory.javaBooleanObjectInspector, + PrimitiveObjectInspectorFactory.writableBooleanObjectInspector, + classOf[BooleanColumnIterator], + true) + assert(builder.stats.min === false) + assert(builder.stats.max === true) + + builder = new BooleanColumnBuilder + builder.setCompressionSchemes(new RLE) + val a = Array.ofDim[java.lang.Boolean](100) + Range(0,100).foreach { i => + a(i) = if (i < 10) true else if (i <80) false else null + } + testColumn( + a, + builder, + PrimitiveObjectInspectorFactory.javaBooleanObjectInspector, + PrimitiveObjectInspectorFactory.writableBooleanObjectInspector, + classOf[BooleanColumnIterator], + true) + } + + test("byte column") { + var builder = new ByteColumnBuilder + testColumn( + Array[java.lang.Byte](1.toByte, 2.toByte, 15.toByte, 55.toByte, 0.toByte, 40.toByte), + builder, + PrimitiveObjectInspectorFactory.javaByteObjectInspector, + PrimitiveObjectInspectorFactory.writableByteObjectInspector, + classOf[ByteColumnIterator]) + assert(builder.stats.min === 0.toByte) + assert(builder.stats.max === 55.toByte) + + builder = new ByteColumnBuilder + testColumn( + Array[java.lang.Byte](null, 2.toByte, 15.toByte, null, 0.toByte, null), + builder, + PrimitiveObjectInspectorFactory.javaByteObjectInspector, + PrimitiveObjectInspectorFactory.writableByteObjectInspector, + classOf[ByteColumnIterator], + true) + assert(builder.stats.min === 0.toByte) + assert(builder.stats.max === 15.toByte) + + builder = new ByteColumnBuilder + builder.setCompressionSchemes(new RLE) + testColumn( + Array[java.lang.Byte](null, 2.toByte, 2.toByte, null, 4.toByte, 4.toByte,4.toByte,5.toByte), + builder, + PrimitiveObjectInspectorFactory.javaByteObjectInspector, + PrimitiveObjectInspectorFactory.writableByteObjectInspector, + classOf[ByteColumnIterator], + true) + } + + test("short column") { + var builder = new ShortColumnBuilder + testColumn( + Array[java.lang.Short](1.toShort, 2.toShort, -15.toShort, 355.toShort, 0.toShort, 40.toShort), + builder, + PrimitiveObjectInspectorFactory.javaShortObjectInspector, + PrimitiveObjectInspectorFactory.writableShortObjectInspector, + classOf[ShortColumnIterator]) + assert(builder.stats.min === -15.toShort) + assert(builder.stats.max === 355.toShort) + + builder = new ShortColumnBuilder + testColumn( + Array[java.lang.Short](1.toShort, 2.toShort, -15.toShort, null, 0.toShort, null), + builder, + PrimitiveObjectInspectorFactory.javaShortObjectInspector, + PrimitiveObjectInspectorFactory.writableShortObjectInspector, + classOf[ShortColumnIterator], + true) + assert(builder.stats.min === -15.toShort) + assert(builder.stats.max === 2.toShort) + + testColumn( + Array[java.lang.Short](1.toShort, 2.toShort, 2.toShort, null, 1.toShort, 1.toShort), + builder, + PrimitiveObjectInspectorFactory.javaShortObjectInspector, + PrimitiveObjectInspectorFactory.writableShortObjectInspector, + classOf[ShortColumnIterator], + true) + } + + test("int column") { + var builder = new IntColumnBuilder + testColumn( + Array[java.lang.Integer](0, 1, 2, 5, 134, -12, 1, 0, 99, 1), + builder, + PrimitiveObjectInspectorFactory.javaIntObjectInspector, + PrimitiveObjectInspectorFactory.writableIntObjectInspector, + classOf[IntColumnIterator]) + assert(builder.stats.min === -12) + assert(builder.stats.max === 134) + + builder = new IntColumnBuilder + testColumn( + Array[java.lang.Integer](null, 1, 2, 5, 134, -12, null, 0, 99, 1), + builder, + PrimitiveObjectInspectorFactory.javaIntObjectInspector, + PrimitiveObjectInspectorFactory.writableIntObjectInspector, + classOf[IntColumnIterator], + true) + assert(builder.stats.min === -12) + assert(builder.stats.max === 134) + + builder = new IntColumnBuilder + builder.setCompressionSchemes(new RLE) + val a = Array.ofDim[java.lang.Integer](100) + Range(0,100).foreach { i => + a(i) = if (i < 10) 10 else if (i <80) 11 else null + } + + testColumn( + a, + builder, + PrimitiveObjectInspectorFactory.javaIntObjectInspector, + PrimitiveObjectInspectorFactory.writableIntObjectInspector, + classOf[IntColumnIterator], + true) + } + + test("long column") { + var builder = new LongColumnBuilder + testColumn( + Array[java.lang.Long](1L, -345345L, 15L, 0L, 23445456L), + builder, + PrimitiveObjectInspectorFactory.javaLongObjectInspector, + PrimitiveObjectInspectorFactory.writableLongObjectInspector, + classOf[LongColumnIterator]) + assert(builder.stats.min === -345345L) + assert(builder.stats.max === 23445456L) + builder = new LongColumnBuilder + testColumn( + Array[java.lang.Long](null, -345345L, 15L, 0L, null), + builder, + PrimitiveObjectInspectorFactory.javaLongObjectInspector, + PrimitiveObjectInspectorFactory.writableLongObjectInspector, + classOf[LongColumnIterator], + true) + assert(builder.stats.min === -345345L) + assert(builder.stats.max === 15L) + + builder = new LongColumnBuilder + builder.setCompressionSchemes(new RLE) + val a = Array.ofDim[java.lang.Long](100) + Range(0,100).foreach { i => + a(i) = if (i < 10) 10 else if (i <80) 11 else null + } + testColumn( + a, + builder, + PrimitiveObjectInspectorFactory.javaLongObjectInspector, + PrimitiveObjectInspectorFactory.writableLongObjectInspector, + classOf[LongColumnIterator], + true) + } + + test("float column") { + var builder = new FloatColumnBuilder + testColumn( + Array[java.lang.Float](1.1.toFloat, -2.5.toFloat, 20000.toFloat, 0.toFloat, 15.0.toFloat), + builder, + PrimitiveObjectInspectorFactory.javaFloatObjectInspector, + PrimitiveObjectInspectorFactory.writableFloatObjectInspector, + classOf[FloatColumnIterator]) + assert(builder.stats.min === -2.5.toFloat) + assert(builder.stats.max === 20000.toFloat) + builder = new FloatColumnBuilder + testColumn( + Array[java.lang.Float](1.1.toFloat, null, 20000.toFloat, null, 15.0.toFloat), + builder, + PrimitiveObjectInspectorFactory.javaFloatObjectInspector, + PrimitiveObjectInspectorFactory.writableFloatObjectInspector, + classOf[FloatColumnIterator], + true) + assert(builder.stats.min === 1.1.toFloat) + assert(builder.stats.max === 20000.toFloat) + } + + test("double column") { + var builder = new DoubleColumnBuilder + testColumn( + Array[java.lang.Double](1.1, 2.2, -2.5, 20000, 0, 15.0), + builder, + PrimitiveObjectInspectorFactory.javaDoubleObjectInspector, + PrimitiveObjectInspectorFactory.writableDoubleObjectInspector, + classOf[DoubleColumnIterator]) + assert(builder.stats.min === -2.5) + assert(builder.stats.max === 20000) + builder = new DoubleColumnBuilder + testColumn( + Array[java.lang.Double](1.1, 2.2, -2.5, null, 0, 15.0), + builder, + PrimitiveObjectInspectorFactory.javaDoubleObjectInspector, + PrimitiveObjectInspectorFactory.writableDoubleObjectInspector, + classOf[DoubleColumnIterator], + true) + assert(builder.stats.min === -2.5) + assert(builder.stats.max === 15.0) + } + + test("string column") { + var builder = new StringColumnBuilder + testColumn( + Array[Text](new Text("a"), new Text(""), new Text("b"), new Text("Abcdz")), + builder, + PrimitiveObjectInspectorFactory.writableStringObjectInspector, + PrimitiveObjectInspectorFactory.writableStringObjectInspector, + classOf[StringColumnIterator], + false, + (a, b) => (a.equals(b.toString)) + ) + assert(builder.stats.min.toString === "") + assert(builder.stats.max.toString === "b") + + builder = new StringColumnBuilder + testColumn( + Array[Text](new Text("a"), new Text(""), null, new Text("b"), new Text("Abcdz"), null), + builder, + PrimitiveObjectInspectorFactory.writableStringObjectInspector, + PrimitiveObjectInspectorFactory.writableStringObjectInspector, + classOf[StringColumnIterator], + false, + (a, b) => if (a == null) b == null else (a.toString.equals(b.toString)) + ) + assert(builder.stats.min.toString === "") + assert(builder.stats.max.toString === "b") + + builder = new StringColumnBuilder + builder.setCompressionSchemes(new RLE) + testColumn( + Array[Text](new Text("a"), new Text("a"), null, new Text("b"), new Text("b"), new Text("Abcdz")), + builder, + PrimitiveObjectInspectorFactory.writableStringObjectInspector, + PrimitiveObjectInspectorFactory.writableStringObjectInspector, + classOf[StringColumnIterator], + false, + (a, b) => if (a == null) b == null else (a.toString.equals(b.toString)) + ) + } + + test("timestamp column") { + val ts1 = new java.sql.Timestamp(0) + val ts2 = new java.sql.Timestamp(500) + ts2.setNanos(400) + val ts3 = new java.sql.Timestamp(1362561610000L) + + var builder = new TimestampColumnBuilder + testColumn( + Array(ts1, ts2, ts3), + builder, + PrimitiveObjectInspectorFactory.javaTimestampObjectInspector, + PrimitiveObjectInspectorFactory.writableTimestampObjectInspector, + classOf[TimestampColumnIterator], + false, + (a, b) => (a.equals(b)) + ) + assert(builder.stats.min.equals(ts1)) + assert(builder.stats.max.equals(ts3)) + + builder = new TimestampColumnBuilder + testColumn( + Array(ts1, ts2, null, ts3, null), + builder, + PrimitiveObjectInspectorFactory.javaTimestampObjectInspector, + PrimitiveObjectInspectorFactory.writableTimestampObjectInspector, + classOf[TimestampColumnIterator], + true, + (a, b) => (a.equals(b)) + ) + assert(builder.stats.min.equals(ts1)) + assert(builder.stats.max.equals(ts3)) + } + + test("Binary Column") { + val b1 = new BytesWritable() + b1.set(Array[Byte](0,1,2), 0, 3) + + val builder = new BinaryColumnBuilder + testColumn( + Array[BytesWritable](b1), + builder, + PrimitiveObjectInspectorFactory.writableBinaryObjectInspector, + PrimitiveObjectInspectorFactory.writableBinaryObjectInspector, + classOf[BinaryColumnIterator], + false, + compareBinary) + assert(builder.stats.isInstanceOf[ColumnStats.NoOpStats[_]]) + + def compareBinary(x: Object, y: Object): Boolean = { + val xdata = x.asInstanceOf[Array[Byte]] + val ywritable = y.asInstanceOf[BytesWritable] + val ydata = ywritable.getBytes() + val length = ywritable.getLength() + if (length != xdata.length) { + false + } else { + val ydatapruned = new Array[Byte](length) + System.arraycopy(ydata, 0, ydatapruned, 0, length) + java.util.Arrays.equals(xdata, ydatapruned) + } + } + } + + + def testColumn[T, U <: ColumnIterator]( + testData: Array[_ <: Object], + builder: ColumnBuilder[T], + oi: ObjectInspector, + writableOi: AbstractPrimitiveWritableObjectInspector, + iteratorClass: Class[U], + expectEWAHWrapper: Boolean = false, + compareFunc: (Object, Object) => Boolean = (a, b) => a == b) { + + builder.initialize(testData.size, "") + testData.foreach { x => builder.append(x, oi)} + val buf = builder.build() + + def executeOneTest() { + val iter = ColumnIterator.newIterator(buf) + + (0 until testData.size).foreach { i => + iter.next() + val expected = testData(i) + val reality = writableOi.getPrimitiveJavaObject(iter.current) + //println ("at position " + i + " expected " + expected + ", but saw " + reality) + assert((expected == null && reality == null) || compareFunc(reality, expected), + "at position " + i + " expected " + expected + ", but saw " + reality) + } + } + + if (PARALLEL_MODE) { + // parallelize to test concurrency + (1 to 10).par.foreach { parallelIndex => executeOneTest() } + } else { + executeOneTest() + } + } +} + diff --git a/src/main/test/scala/shark/memstore2/ColumnStatsSuite.scala b/src/main/test/scala/shark/memstore2/ColumnStatsSuite.scala new file mode 100755 index 00000000..e1e51402 --- /dev/null +++ b/src/main/test/scala/shark/memstore2/ColumnStatsSuite.scala @@ -0,0 +1,232 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2 + +import java.sql.Timestamp +import scala.language.implicitConversions + +import org.apache.hadoop.io.Text + +import org.scalatest.FunSuite + +import shark.memstore2.column.ColumnStats + + +class ColumnStatsSuite extends FunSuite { + + test("BooleanColumnStats") { + var c = new ColumnStats.BooleanColumnStats + c.append(false) + assert(c.min == false && c.max == false) + c.append(false) + assert(c.min == false && c.max == false) + c.append(true) + assert(c.min == false && c.max == true) + + c = new ColumnStats.BooleanColumnStats + c.append(true) + assert(c.min == true && c.max == true) + c.append(false) + assert(c.min == false && c.max == true) + assert(c := true) + assert(c := false) + c = new ColumnStats.BooleanColumnStats + c.append(true) + assert(c := true) + assert(!(c := false)) + assert(c :>= false) + assert(!(c :<= false)) + assert(c :>= true) + } + + test("ByteColumnStats") { + val c = new ColumnStats.ByteColumnStats + c.append(0) + assert(c.min == 0 && c.max == 0) + c.append(1) + assert(c.min == 0 && c.max == 1) + c.append(-1) + assert(c.min == -1 && c.max == 1) + c.append(2) + assert(c.min == -1 && c.max == 2) + c.append(-2) + assert(c.min == -2 && c.max == 2) + assert(c := 0.toByte) + assert(c :> 0.toByte) + assert(c :<= -1.toByte) + assert(!(c :<= -3.toByte)) + } + + test("ShortColumnStats") { + val c = new ColumnStats.ShortColumnStats + c.append(0) + assert(c.min == 0 && c.max == 0) + c.append(1) + assert(c.min == 0 && c.max == 1) + c.append(-1) + assert(c.min == -1 && c.max == 1) + c.append(1024) + assert(c.min == -1 && c.max == 1024) + c.append(-1024) + assert(c.min == -1024 && c.max == 1024) + } + + test("IntColumnStats") { + var c = new ColumnStats.IntColumnStats + c.append(0) + assert(c.min == 0 && c.max == 0) + c.append(1) + assert(c.min == 0 && c.max == 1) + c.append(-1) + assert(c.min == -1 && c.max == 1) + c.append(65537) + assert(c.min == -1 && c.max == 65537) + c.append(-65537) + assert(c.min == -65537 && c.max == 65537) + + c = new ColumnStats.IntColumnStats + assert(c.isOrdered && c.isAscending && c.isDescending) + assert(c.maxDelta == 0) + + c = new ColumnStats.IntColumnStats + Array(1).foreach(c.append) + assert(c.isOrdered && c.isAscending && c.isDescending) + assert(c.maxDelta == 0) + + c = new ColumnStats.IntColumnStats + Array(1, 2, 3, 3, 4, 22).foreach(c.append) + assert(c.isOrdered && c.isAscending && !c.isDescending) + assert(c.maxDelta == 18) + + c = new ColumnStats.IntColumnStats + Array(22, 1, 0, -5).foreach(c.append) + assert(c.isOrdered && !c.isAscending && c.isDescending) + assert(c.maxDelta == 21) + + c = new ColumnStats.IntColumnStats + Array(22, 1, 24).foreach(c.append) + assert(!c.isOrdered && !c.isAscending && !c.isDescending) + + c = new ColumnStats.IntColumnStats + Array(22, 1, 24).foreach(c.append) + assert(c := 13 === true) + assert(c :> 13 === true) + assert(c :< 13 === true) + assert(c := 0 === false) + assert(c :> 25 === false) + assert(c :< 1 === false) + } + + test("LongColumnStats") { + val c = new ColumnStats.LongColumnStats + c.append(0) + assert(c.min == 0 && c.max == 0) + c.append(1) + assert(c.min == 0 && c.max == 1) + c.append(-1) + assert(c.min == -1 && c.max == 1) + c.append(Int.MaxValue.toLong + 1L) + assert(c.min == -1 && c.max == Int.MaxValue.toLong + 1L) + c.append(Int.MinValue.toLong - 1L) + assert(c.min == Int.MinValue.toLong - 1L && c.max == Int.MaxValue.toLong + 1L) + assert(c := 0.toLong) + assert(c :> -2.toLong) + assert(c :< 0.toLong) + } + + test("FloatColumnStats") { + val c = new ColumnStats.FloatColumnStats + c.append(0) + assert(c.min == 0 && c.max == 0) + c.append(1) + assert(c.min == 0 && c.max == 1) + c.append(-1) + assert(c.min == -1 && c.max == 1) + c.append(20.5445F) + assert(c.min == -1 && c.max == 20.5445F) + c.append(-20.5445F) + assert(c.min == -20.5445F && c.max == 20.5445F) + assert(c := 20.5F) + assert(c :< 20.6F) + assert(c :> -20.6F) + } + + test("DoubleColumnStats") { + val c = new ColumnStats.DoubleColumnStats + c.append(0) + assert(c.min == 0 && c.max == 0) + c.append(1) + assert(c.min == 0 && c.max == 1) + c.append(-1) + assert(c.min == -1 && c.max == 1) + c.append(20.5445) + assert(c.min == -1 && c.max == 20.5445) + c.append(-20.5445) + assert(c.min == -20.5445 && c.max == 20.5445) + assert(c := 20.5) + assert(!(c :> 20.6)) + assert(c :< 20.6) + } + + test("TimestampColumnStats") { + val c = new ColumnStats.TimestampColumnStats + val ts1 = new Timestamp(1000) + val ts2 = new Timestamp(2000) + val ts3 = new Timestamp(1500) + val ts4 = new Timestamp(2000) + ts4.setNanos(100) + c.append(ts1) + + assert(c.min.equals(ts1) && c.max.equals(ts1)) + c.append(ts2) + assert(c.min.equals(ts1) && c.max.equals(ts2)) + c.append(ts3) + assert(c.min.equals(ts1) && c.max.equals(ts2)) + + + assert(c.min.equals(ts1) && c.max.equals(ts2)) + c.append(ts4) + assert(c.min.equals(ts1) && c.max.equals(ts4)) + + } + + test("StringColumnStats") { + implicit def T(str: String): Text = new Text(str) + val c = new ColumnStats.StringColumnStats + assert(c.min == null && c.max == null) + + assert(!(c :> "test")) + assert(!(c :< "test")) + assert(!(c == "test")) + + c.append("a") + assert(c.min.equals(T("a")) && c.max.equals(T("a"))) + + assert(c.min.equals(T("a")) && c.max.equals(T("a"))) + c.append("b") + assert(c.min.equals(T("a")) && c.max.equals(T("b"))) + c.append("b") + assert(c.min.equals(T("a")) && c.max.equals(T("b"))) + c.append("cccc") + + assert(c.min.equals(T("a")) && c.max.equals(T("cccc"))) + c.append("0987") + assert(c.min.equals(T("0987")) && c.max.equals(T("cccc"))) + + } +} diff --git a/src/main/test/scala/shark/memstore2/TablePartitionSuite.scala b/src/main/test/scala/shark/memstore2/TablePartitionSuite.scala new file mode 100755 index 00000000..fb91027f --- /dev/null +++ b/src/main/test/scala/shark/memstore2/TablePartitionSuite.scala @@ -0,0 +1,121 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2 + +import java.nio.ByteBuffer + +import org.scalatest.FunSuite + +import org.apache.spark.SparkConf +import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} + + +class TablePartitionSuite extends FunSuite { + + test("serialize TablePartition backed by non-direct ByteBuffer using Java") { + val col1 = Array[Byte](0, 1, 2) + val col2 = Array[Byte](1, 2, 3) + val tp = new TablePartition(3, Array(ByteBuffer.wrap(col1), ByteBuffer.wrap(col2))) + + val ser = new JavaSerializer(new SparkConf(false)) + val bytes = ser.newInstance().serialize(tp) + val tp1 = ser.newInstance().deserialize[TablePartition](bytes) + assert(tp1.numRows === 3) + assert(tp1.columns(0).remaining() == 3) + assert(tp1.columns(0).get() == 0) + assert(tp1.columns(0).get() == 1) + assert(tp1.columns(0).get() == 2) + assert(tp1.columns(1).remaining() == 3) + assert(tp1.columns(1).get() == 1) + assert(tp1.columns(1).get() == 2) + assert(tp1.columns(1).get() == 3) + } + + test("serialize TablePartition backed by direct ByteBuffer using Java") { + val col1 = ByteBuffer.allocateDirect(3) + col1.put(0.toByte) + col1.put(1.toByte) + col1.put(2.toByte) + col1.rewind() + val col2 = ByteBuffer.allocateDirect(3) + col2.put(1.toByte) + col2.put(2.toByte) + col2.put(3.toByte) + col2.rewind() + val tp = new TablePartition(3, Array(col1, col2)) + + val ser = new JavaSerializer(new SparkConf(false)) + val bytes = ser.newInstance().serialize(tp) + val tp1 = ser.newInstance().deserialize[TablePartition](bytes) + assert(tp1.numRows === 3) + assert(tp1.columns(0).remaining() == 3) + assert(tp1.columns(0).get() == 0) + assert(tp1.columns(0).get() == 1) + assert(tp1.columns(0).get() == 2) + assert(tp1.columns(1).remaining() == 3) + assert(tp1.columns(1).get() == 1) + assert(tp1.columns(1).get() == 2) + assert(tp1.columns(1).get() == 3) + } + + test("serialize TablePartition backed by non-direct ByteBuffer using Kryo") { + val col1 = Array[Byte](0, 1, 2) + val col2 = Array[Byte](1, 2, 3) + val tp = new TablePartition(3, Array(ByteBuffer.wrap(col1), ByteBuffer.wrap(col2))) + + val ser = new KryoSerializer(new SparkConf(false)) + val bytes = ser.newInstance().serialize(tp) + val tp1 = ser.newInstance().deserialize[TablePartition](bytes) + assert(tp1.numRows === 3) + assert(tp1.columns(0).remaining() == 3) + assert(tp1.columns(0).get() == 0) + assert(tp1.columns(0).get() == 1) + assert(tp1.columns(0).get() == 2) + assert(tp1.columns(1).remaining() == 3) + assert(tp1.columns(1).get() == 1) + assert(tp1.columns(1).get() == 2) + assert(tp1.columns(1).get() == 3) + } + + test("serialize TablePartition backed by direct ByteBuffer using Kryo") { + val col1 = ByteBuffer.allocateDirect(3) + col1.put(0.toByte) + col1.put(1.toByte) + col1.put(2.toByte) + col1.rewind() + val col2 = ByteBuffer.allocateDirect(3) + col2.put(1.toByte) + col2.put(2.toByte) + col2.put(3.toByte) + col2.rewind() + val tp = new TablePartition(3, Array(col1, col2)) + + val ser = new KryoSerializer(new SparkConf(false)) + val bytes = ser.newInstance().serialize(tp) + val tp1 = ser.newInstance().deserialize[TablePartition](bytes) + assert(tp1.numRows === 3) + assert(tp1.columns(0).remaining() == 3) + assert(tp1.columns(0).get() == 0) + assert(tp1.columns(0).get() == 1) + assert(tp1.columns(0).get() == 2) + assert(tp1.columns(1).remaining() == 3) + assert(tp1.columns(1).get() == 1) + assert(tp1.columns(1).get() == 2) + assert(tp1.columns(1).get() == 3) + } +} diff --git a/src/main/test/scala/shark/memstore2/column/ColumnTypeSuite.scala b/src/main/test/scala/shark/memstore2/column/ColumnTypeSuite.scala new file mode 100755 index 00000000..404a01b1 --- /dev/null +++ b/src/main/test/scala/shark/memstore2/column/ColumnTypeSuite.scala @@ -0,0 +1,121 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2.column + +import java.nio.ByteBuffer + +import org.apache.hadoop.io.IntWritable +import org.apache.hadoop.io.LongWritable +import org.apache.hadoop.hive.serde2.io._ + +import org.scalatest.FunSuite + +class ColumnTypeSuite extends FunSuite { + + test("Int") { + assert(INT.defaultSize == 4) + var buffer = ByteBuffer.allocate(32) + var a: Seq[Int] = Array[Int](35, 67, 899, 4569001) + a.foreach {i => buffer.putInt(i)} + buffer.rewind() + a.foreach {i => + val v = INT.extract(buffer) + assert(v == i) + } + buffer = ByteBuffer.allocate(32) + a = Range(0, 4) + a.foreach { i => + INT.append(i, buffer) + } + buffer.rewind() + a.foreach { i => assert(buffer.getInt() == i)} + + buffer = ByteBuffer.allocate(32) + a =Range(0,4) + a.foreach { i => buffer.putInt(i)} + buffer.rewind() + val writable = new IntWritable() + a.foreach { i => + INT.extractInto(buffer, writable) + assert(writable.get == i) + } + + } + + test("Short") { + assert(SHORT.defaultSize == 2) + assert(SHORT.actualSize(8) == 2) + var buffer = ByteBuffer.allocate(32) + var a = Array[Short](35, 67, 87, 45) + a.foreach {i => buffer.putShort(i)} + buffer.rewind() + a.foreach {i => + val v = SHORT.extract(buffer) + assert(v == i) + } + + buffer = ByteBuffer.allocate(32) + a = Array[Short](0,1,2,3) + a.foreach { i => + SHORT.append(i, buffer) + } + buffer.rewind() + a.foreach { i => assert(buffer.getShort() == i)} + + buffer = ByteBuffer.allocate(32) + a =Array[Short](0,1,2,3) + a.foreach { i => buffer.putShort(i)} + buffer.rewind() + val writable = new ShortWritable() + a.foreach { i => + SHORT.extractInto(buffer, writable) + assert(writable.get == i) + } + } + + test("Long") { + assert(LONG.defaultSize == 8) + assert(LONG.actualSize(45L) == 8) + var buffer = ByteBuffer.allocate(64) + var a = Array[Long](35L, 67L, 8799000880L, 45000999090L) + a.foreach {i => buffer.putLong(i)} + buffer.rewind() + a.foreach {i => + val v = LONG.extract(buffer) + assert(v == i) + } + + buffer = ByteBuffer.allocate(32) + a = Array[Long](0,1,2,3) + a.foreach { i => + LONG.append(i, buffer) + } + buffer.rewind() + a.foreach { i => assert(buffer.getLong() == i)} + + buffer = ByteBuffer.allocate(32) + a =Array[Long](0,1,2,3) + a.foreach { i => buffer.putLong(i)} + buffer.rewind() + val writable = new LongWritable() + a.foreach { i => + LONG.extractInto(buffer, writable) + assert(writable.get == i) + } + } +} diff --git a/src/main/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala b/src/main/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala new file mode 100755 index 00000000..d891ec13 --- /dev/null +++ b/src/main/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala @@ -0,0 +1,243 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2.column + +import java.nio.ByteBuffer +import java.nio.ByteOrder +import org.scalatest.FunSuite + +import org.apache.hadoop.io.Text +import org.apache.hadoop.hive.serde2.objectinspector.primitive._ +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector + +import shark.memstore2.column.Implicits._ + +class CompressedColumnIteratorSuite extends FunSuite { + + /** + * Generic tester across types and encodings. The function applies the given compression + * algorithm on the given sequence of values, and test whether the resulting iterator gives + * the same sequence of values. + * + * If we expect the compression algorithm to not compress the data, we should set the + * shouldNotCompress flag to true. This way, it doesn't actually create a compressed buffer, + * but simply tests the compression ratio returned by the algorithm is >= 1.0. + */ + def testList[T, W]( + l: Seq[T], + t: ColumnType[T, _], + algo: CompressionAlgorithm, + expectedCompressedSize: Long, + shouldNotCompress: Boolean = false) + { + val b = ByteBuffer.allocate(1024 + (3 * 40 * l.size)) + b.order(ByteOrder.nativeOrder()) + b.putInt(t.typeID) + l.foreach { item => + t.append(item, b) + algo.gatherStatsForCompressibility(item, t.asInstanceOf[ColumnType[Any, _]]) + } + b.limit(b.position()) + b.rewind() + + info("compressed size: %d, uncompressed size: %d, compression ratio %f".format( + algo.compressedSize, algo.uncompressedSize, algo.compressionRatio)) + + info("expected compressed size: %d".format(expectedCompressedSize)) + assert(algo.compressedSize === expectedCompressedSize) + + if (shouldNotCompress) { + assert(algo.compressionRatio >= 1.0) + } else { + val compressedBuffer = algo.compress(b, t) + val iter = new TestIterator(compressedBuffer, compressedBuffer.getInt()) + + val oi: ObjectInspector = t match { + case BOOLEAN => PrimitiveObjectInspectorFactory.writableBooleanObjectInspector + case BYTE => PrimitiveObjectInspectorFactory.writableByteObjectInspector + case SHORT => PrimitiveObjectInspectorFactory.writableShortObjectInspector + case INT => PrimitiveObjectInspectorFactory.writableIntObjectInspector + case LONG => PrimitiveObjectInspectorFactory.writableLongObjectInspector + case STRING => PrimitiveObjectInspectorFactory.writableStringObjectInspector + case _ => throw new UnsupportedOperationException("Unsupported compression type " + t) + } + + l.foreach { x => + assert(iter.hasNext) + iter.next() + assert(t.get(iter.current, oi) === x) + } + + // Make sure we reach the end of the iterator. + assert(!iter.hasNext) + } + } + + test("RLE Boolean") { + // 3 runs: (1+4)*3 + val bools = Seq(true, true, false, true, true, true, true, true, true, true, true, true) + testList(bools, BOOLEAN, new RLE, 15) + } + + test("RLE Byte") { + // 3 runs: (1+4)*3 + testList(Seq[Byte](10, 10, 10, 10, 10, 10, 10, 10, 10, 20, 10), BYTE, new RLE, 15) + } + + test("RLE Short") { + // 3 runs: (2+4)*3 + testList(Seq[Short](10, 10, 10, 20000, 20000, 20000, 500, 500, 500, 500), SHORT, new RLE, 18) + } + + test("RLE Int") { + // 3 runs: (4+4)*3 + testList(Seq[Int](1000000, 1000000, 1000000, 1000000, 900000, 99), INT, new RLE, 24) + } + + test("RLE Long") { + // 2 runs: (8+4)*3 + val longs = Seq[Long](2147483649L, 2147483649L, 2147483649L, 2147483649L, 500L, 500L, 500L) + testList(longs, LONG, new RLE, 24) + } + + test("RLE String") { + // 3 runs: (4+4+4) + (4+1+4) + (4+1+4) = 30 + val strs: Seq[Text] = Seq("abcd", "abcd", "abcd", "e", "e", "!", "!").map(s => new Text(s)) + testList(strs, STRING, new RLE, 30) + } + + test("Dictionary Encoded Int") { + // dict len + 3 distinct values + 7 values = 4 + 3*4 + 7*2 = 30 + val ints = Seq[Int](1000000, 1000000, 99, 1000000, 1000000, 900000, 99) + testList(ints, INT, new DictionaryEncoding, 30) + } + + test("Dictionary Encoded Long") { + // dict len + 2 distinct values + 7 values = 4 + 2*8 + 7*2 = 34 + val longs = Seq[Long](2147483649L, 2147483649L, 2147483649L, 2147483649L, 500L, 500L, 500L) + testList(longs, LONG, new DictionaryEncoding, 34) + } + + test("Dictionary Encoded String") { + // dict len + 3 distinct values + 8 values = 4 + (4+4) + (4+1) + (4+1) + 8*2 = + val strs: Seq[Text] = Seq("abcd", "abcd", "abcd", "e", "e", "e", "!", "!").map(s => new Text(s)) + testList(strs, STRING, new DictionaryEncoding, 38, shouldNotCompress = false) + } + + test("Dictionary Encoding at limit of unique values") { + val ints = Range(0, Short.MaxValue - 1).flatMap(i => Iterator(i, i, i)) + val expectedLen = 4 + (Short.MaxValue - 1) * 4 + 2 * (Short.MaxValue - 1) * 3 + testList(ints, INT, new DictionaryEncoding, expectedLen) + } + + test("Dictionary Encoding - should not compress") { + val ints = Range(0, Short.MaxValue.toInt) + testList(ints, INT, new DictionaryEncoding, Int.MaxValue, shouldNotCompress = true) + } + + test("RLE - should not compress") { + val ints = Range(0, Short.MaxValue.toInt + 1) + val expectedLen = (Short.MaxValue.toInt + 1) * (4 + 4) + testList(ints, INT, new RLE, expectedLen, shouldNotCompress = true) + } + + test("BooleanBitSet Boolean (shorter)") { + // 1 Long worth of Booleans, in addition to the length field: 4+8 + val bools = Seq(true, true, false, false) + testList(bools, BOOLEAN, new BooleanBitSetCompression, 4+8) + } + + test("BooleanBitSet Boolean (longer)") { + // 2 Longs worth of Booleans, in addition to the length field: 4+8+8 + val bools = Seq(true, true, false, false, true, true, false, false,true, true, false, false,true, true, false, false, + true, true, false, false,true, true, false, false, true, true, false, false,true, true, false, false, + true, true, false, false,true, true, false, false, true, true, false, false,true, true, false, false, + true, true, false, false,true, true, false, false, true, true, false, false,true, true, false, false, + true, true, false, false,true, true, false, false, true, true, false, false,true, true, false, false) + testList(bools, BOOLEAN, new BooleanBitSetCompression, 4+8+8) + } + + test("BooleanBitSet Boolean should not compress - compression ratio > 1") { + // 1 Long worth of Booleans, in addtion to the length field: 4+8 + val bools = Seq(true, false) + testList(bools, BOOLEAN, new BooleanBitSetCompression, 4+8, shouldNotCompress = true) + } + + test("IntDeltaEncoding") { + // base 5 + 4 small diffs + newBase 5 = 14 + val ints = Seq[Int](1000000, 1000001, 1000002, 1000003, 1000004, 5) + testList(ints, INT, new IntDeltaEncoding, 5 + 4 + 5) + + val ints2 = Seq[Int](1000000, 1000001, 1000000, 1000004, 1000001, 5) + testList(ints2, INT, new IntDeltaEncoding, 5 + 4 + 5) + + testList(List(0, 62), INT, new IntDeltaEncoding, 1 + 4 + 1) + testList(List(0, 63), INT, new IntDeltaEncoding, 1 + 4 + 1) + testList(List(0, 64), INT, new IntDeltaEncoding, 1 + 4 + 1) + testList(List(0, 63, 64), INT, new IntDeltaEncoding, 1 + 4 + 1 + 1) + testList(List(0, 128, -125), INT, new IntDeltaEncoding, 1 + 4 + 1 + 4 + 1 + 4) + + testList(List(0, 12400, 12600, 100, 228), INT, new IntDeltaEncoding, 5 * 5) + testList(Range(-4, 0), INT, new IntDeltaEncoding, 1 + 4 + 3) + + val ints3 = Range(0, Byte.MaxValue.toInt - 1) + testList(ints3, INT, new IntDeltaEncoding, 1 + 4 + 125) + + val ints4 = Range(Byte.MinValue.toInt + 2, 0) + testList(ints4, INT, new IntDeltaEncoding, 1 + 4 + 125) + } + + test("LongDeltaEncoding") { + // base 9 + 3 small deltas + newBase 9 + 2 small deltas = 23 + val longs = Seq[Long](2147483649L, 2147483649L, 2147483649L, 2147483649L, 500L, 500L, 500L) + testList(longs, LONG, new LongDeltaEncoding, 23) + } + + test("int delta encoding boundary condition") { + // 127 deltas are fine, while 128 are not. + var skips = Range(0, 1000).map { x => -127 * x } + testList(skips, INT, new IntDeltaEncoding, 1 + 4 + 999) + + skips = Range(0, 1000).map { x => 127 * x } + testList(skips, INT, new IntDeltaEncoding, 1 + 4 + 999) + + skips = Range(0, 1000).map { x => 128 * x } + testList(skips, INT, new IntDeltaEncoding, 1 + 4 + (5 * 999)) + + skips = Range(0, 1000).map { x => -128 * x } + testList(skips, INT, new IntDeltaEncoding, 1 + 4 + (5 * 999)) + } + + test("long delta encoding boundary condition") { + // 127 deltas are fine, while 128 are not. + var skips = Range(0, 1000).map { x => (-127 * x).toLong } + testList(skips, LONG, new LongDeltaEncoding, 1 + 8 + 999) + + skips = Range(0, 1000).map { x => (127 * x).toLong } + testList(skips, LONG, new LongDeltaEncoding, 1 + 8 + 999) + + skips = Range(0, 1000).map { x => (128 * x).toLong } + testList(skips, LONG, new LongDeltaEncoding, 1 + 8 + (9 * 999)) + + skips = Range(0, 1000).map { x => (-128 * x).toLong } + testList(skips, LONG, new LongDeltaEncoding, 1 + 8 + (9 * 999)) + } +} + +class TestIterator(val buffer: ByteBuffer, val columnType: ColumnType[_,_]) + extends CompressedColumnIterator diff --git a/src/main/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala b/src/main/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala new file mode 100755 index 00000000..7f7a118e --- /dev/null +++ b/src/main/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala @@ -0,0 +1,366 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2.column + +import java.nio.{ByteBuffer, ByteOrder} + +import scala.collection.mutable.HashMap + +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory +import org.apache.hadoop.io.Text + +import org.scalatest.FunSuite + +import shark.memstore2.column.ColumnStats._ + +class CompressionAlgorithmSuite extends FunSuite { + + // TODO: clean these tests. + + test("CompressedColumnBuilder using RLE") { + + class TestColumnBuilder(val stats: ColumnStats[Int], val t: ColumnType[Int,_]) + extends CompressedColumnBuilder[Int] { + override def shouldApply(scheme: CompressionAlgorithm) = true + } + + val b = new TestColumnBuilder(new NoOpStats, INT) + b.setCompressionSchemes(new RLE) + b.initialize(100) + val oi = PrimitiveObjectInspectorFactory.javaIntObjectInspector + b.append(123.asInstanceOf[Object], oi) + b.append(123.asInstanceOf[Object], oi) + b.append(56.asInstanceOf[Object], oi) + b.append(56.asInstanceOf[Object], oi) + val compressedBuffer = b.build() + assert(compressedBuffer.getInt() == INT.typeID) + assert(compressedBuffer.getInt() == RLECompressionType.typeID) + + assert(compressedBuffer.getInt() == 123) + assert(compressedBuffer.getInt() == 2) + assert(compressedBuffer.getInt() == 56) + assert(compressedBuffer.getInt() == 2) + assert(!compressedBuffer.hasRemaining) + } + + test("RLE Strings") { + val b = ByteBuffer.allocate(1024) + b.order(ByteOrder.nativeOrder()) + b.putInt(STRING.typeID) + val rle = new RLE() + Seq[Text](new Text("abc"), new Text("abc"), new Text("efg"), new Text("abc")).foreach { text => + STRING.append(text, b) + rle.gatherStatsForCompressibility(text, STRING) + } + b.limit(b.position()) + b.rewind() + val compressedBuffer = rle.compress(b, STRING) + assert(compressedBuffer.getInt() == STRING.typeID) + assert(compressedBuffer.getInt() == RLECompressionType.typeID) + assert(STRING.extract(compressedBuffer).equals(new Text("abc"))) + assert(compressedBuffer.getInt() == 2) + assert(STRING.extract(compressedBuffer).equals(new Text("efg"))) + assert(compressedBuffer.getInt() == 1) + assert(STRING.extract(compressedBuffer).equals(new Text("abc"))) + assert(compressedBuffer.getInt() == 1) + assert(!compressedBuffer.hasRemaining) + } + + test("RLE int with run length 1") { + val b = ByteBuffer.allocate(16) + b.order(ByteOrder.nativeOrder()) + b.putInt(INT.typeID) + b.putInt(123) + b.putInt(56) + b.limit(b.position()) + b.rewind() + val rle = new RLE() + rle.gatherStatsForCompressibility(123, INT) + rle.gatherStatsForCompressibility(56, INT) + val compressedBuffer = rle.compress(b, INT) + assert(compressedBuffer.getInt() == INT.typeID) + assert(compressedBuffer.getInt() == RLECompressionType.typeID) + assert(compressedBuffer.getInt() == 123) + assert(compressedBuffer.getInt() == 1) + assert(compressedBuffer.getInt() == 56) + assert(compressedBuffer.getInt() == 1) + assert(!compressedBuffer.hasRemaining) + } + + test("RLE int single run") { + val b = ByteBuffer.allocate(4008) + b.order(ByteOrder.nativeOrder()) + b.putInt(INT.typeID) + val rle = new RLE() + Range(0, 1000).foreach { x => + b.putInt(6) + rle.gatherStatsForCompressibility(6, INT) + } + b.limit(b.position()) + b.rewind() + val compressedBuffer = rle.compress(b, INT) + assert(compressedBuffer.getInt() == INT.typeID) + assert(compressedBuffer.getInt() == RLECompressionType.typeID) + assert(compressedBuffer.getInt() == 6) + assert(compressedBuffer.getInt() == 1000) + assert(!compressedBuffer.hasRemaining) + } + + test("RLE long single run") { + val b = ByteBuffer.allocate(8008) + b.order(ByteOrder.nativeOrder()) + b.putInt(LONG.typeID) + val rle = new RLE() + Range(0, 1000).foreach { x => + b.putLong(Long.MaxValue - 6) + rle.gatherStatsForCompressibility(Long.MaxValue - 6, LONG) + } + b.limit(b.position()) + b.rewind() + val compressedBuffer = rle.compress(b, LONG) + assert(compressedBuffer.getInt() == LONG.typeID) + assert(compressedBuffer.getInt() == RLECompressionType.typeID) + assert(compressedBuffer.getLong() == Long.MaxValue - 6) + assert(compressedBuffer.getInt() == 1000) + assert(!compressedBuffer.hasRemaining) + } + + test("RLE int 3 runs") { + val b = ByteBuffer.allocate(4008) + b.order(ByteOrder.nativeOrder()) + b.putInt(INT.typeID) + val items = Array[Int](10, 20, 40) + val rle = new RLE() + + Range(0, 1000).foreach { x => + val v = if (x < 100) items(0) else if (x < 500) items(1) else items(2) + b.putInt(v) + rle.gatherStatsForCompressibility(v, INT) + } + b.limit(b.position()) + b.rewind() + val compressedBuffer = rle.compress(b, INT) + assert(compressedBuffer.getInt() == INT.typeID) + assert(compressedBuffer.getInt() == RLECompressionType.typeID) + assert(compressedBuffer.getInt() == 10) + assert(compressedBuffer.getInt() == 100) + assert(compressedBuffer.getInt() == 20) + assert(compressedBuffer.getInt() == 400) + assert(compressedBuffer.getInt() == 40) + assert(compressedBuffer.getInt() == 500) + assert(!compressedBuffer.hasRemaining) + } + + test("RLE int single long run") { + val b = ByteBuffer.allocate(4000008) + b.order(ByteOrder.nativeOrder()) + b.putInt(INT.typeID) + val rle = new RLE() + + Range(0, 1000000).foreach { x => + b.putInt(6) + rle.gatherStatsForCompressibility(6, INT) + } + b.limit(b.position()) + b.rewind() + val compressedBuffer = rle.compress(b, INT) + assert(compressedBuffer.getInt() == RLECompressionType.typeID) + assert(compressedBuffer.getInt() == INT.typeID) + assert(compressedBuffer.getInt() == 6) + assert(compressedBuffer.getInt() == 1000000) + assert(!compressedBuffer.hasRemaining) + } + + test("IntDeltaEncoding") { + val b = ByteBuffer.allocate(1024) + b.order(ByteOrder.nativeOrder()) + b.putInt(INT.typeID) + + val bde = new IntDeltaEncoding + + val x = 1 + b.putInt(x) + bde.gatherStatsForCompressibility(x, INT) + + val y = x + 40000 + b.putInt(y) + bde.gatherStatsForCompressibility(y, INT) + + val z = y + 1 + b.putInt(z) + bde.gatherStatsForCompressibility(z, INT) + + b.limit(b.position()) + b.rewind() + val compressedBuffer = bde.compress(b, INT) + assert(compressedBuffer.getInt() == INT.typeID) + assert(compressedBuffer.getInt() == IntDeltaCompressionType.typeID) + + compressedBuffer.get() // first flagByte + assert(INT.extract(compressedBuffer).equals(x)) + + compressedBuffer.get() // second flagByte + assert(INT.extract(compressedBuffer).equals(y)) + + val seven: Byte = compressedBuffer.get() // third flagByte + assert(seven === 1.toByte) + + assert(!compressedBuffer.hasRemaining) + } + + test("LongDeltaEncoding") { + val b = ByteBuffer.allocate(10024) + b.order(ByteOrder.nativeOrder()) + b.putInt(LONG.typeID) + + val bde = new LongDeltaEncoding + + val x: Long = 1 + b.putLong(x) + bde.gatherStatsForCompressibility(x, LONG) + + val y: Long = x + 40000 + b.putLong(y) + bde.gatherStatsForCompressibility(y, LONG) + + val z: Long = y + 1 + b.putLong(z) + bde.gatherStatsForCompressibility(z, LONG) + + b.limit(b.position()) + b.rewind() + val compressedBuffer = bde.compress(b, LONG) + assert(compressedBuffer.getInt() === LONG.typeID) + assert(compressedBuffer.getInt() === LongDeltaCompressionType.typeID) + + compressedBuffer.get() // first flagByte + assert(LONG.extract(compressedBuffer).equals(x)) + + compressedBuffer.get() // second flagByte + assert(LONG.extract(compressedBuffer).equals(y)) + + val seven: Byte = compressedBuffer.get() // third flagByte + assert(seven === 1.toByte) + + assert(!compressedBuffer.hasRemaining) + } + + test("Dictionary Encoding") { + + def testList[T]( + l: Seq[T], + u: ColumnType[T, _], + expectedDictSize: Int, + compareFunc: (T, T) => Boolean = (a: T, b: T) => a == b) { + + val b = ByteBuffer.allocate(1024 + (3*40*l.size)) + b.order(ByteOrder.nativeOrder()) + b.putInt(u.typeID) + val de = new DictionaryEncoding() + l.foreach { item => + assert(de.supportsType(u)) + u.append(item, b) + de.gatherStatsForCompressibility(item, u) + } + b.limit(b.position()) + b.rewind() + val compressedBuffer = de.compress(b, u) + assert(compressedBuffer.getInt() === u.typeID) + assert(compressedBuffer.getInt() === DictionaryCompressionType.typeID) + assert(compressedBuffer.getInt() === expectedDictSize) //dictionary size + val dictionary = new HashMap[Short, T]() + var count = 0 + while (count < expectedDictSize) { + val v = u.extract(compressedBuffer) + dictionary.put(dictionary.size.toShort, u.clone(v)) + count += 1 + } + assert(dictionary.get(0).get.equals(l(0))) + assert(dictionary.get(1).get.equals(l(2))) + l.foreach { x => + val y = dictionary.get(compressedBuffer.getShort()).get + assert(compareFunc(y, x)) + } + } + + val iList = Array[Int](10, 10, 20, 10) + val lList = iList.map { i => Long.MaxValue - i.toLong } + val sList = iList.map { i => new Text(i.toString) } + + testList(iList, INT, 2) + testList(lList, LONG, 2) + testList(sList, STRING, 2, (a: Text, b: Text) => a.hashCode == b.hashCode) + + // test at limit of unique values + val alternating = Range(0, Short.MaxValue-1, 1).flatMap { s => List(1, s) } + val longList = List.concat(iList, alternating, iList) + assert(longList.size === (8 + 2*(Short.MaxValue-1))) + testList(longList, INT, Short.MaxValue - 1) + } + + test("Uncompressed text") { + val b = new StringColumnBuilder + b.initialize(0) + val oi = PrimitiveObjectInspectorFactory.javaStringObjectInspector + + val lines = Array[String]( + "lar deposits. blithely final packages cajole. regular waters are final requests.", + "hs use ironic, even requests. s", + "ges. thinly even pinto beans ca", + "ly final courts cajole furiously final excuse", + "uickly special accounts cajole carefully blithely close requests. carefully final" + ) + lines.foreach { line => + b.append(line, oi) + } + val newBuffer = b.build() + assert(newBuffer.getInt() === 0) // null count + assert(newBuffer.getInt() === STRING.typeID) + assert(newBuffer.getInt() === DefaultCompressionType.typeID) + } + + test("BooleanBitSet encoding") { + val bbs = new BooleanBitSetCompression() + val b = ByteBuffer.allocate(4 + 64 + 2) + b.order(ByteOrder.nativeOrder()) + b.putInt(BOOLEAN.typeID) + for(_ <- 1 to 5) { + b.put(0.toByte) + b.put(1.toByte) + bbs.gatherStatsForCompressibility(false, BOOLEAN) + bbs.gatherStatsForCompressibility(true, BOOLEAN) + } + for(_ <- 1 to 54) { + b.put(0.toByte) + bbs.gatherStatsForCompressibility(false, BOOLEAN) + } + b.put(0.toByte) + b.put(1.toByte) + bbs.gatherStatsForCompressibility(false, BOOLEAN) + bbs.gatherStatsForCompressibility(true, BOOLEAN) + b.limit(b.position()) + b.rewind() + val compressedBuffer = bbs.compress(b, BOOLEAN) + assert(compressedBuffer.getInt() === BOOLEAN.typeID) + assert(compressedBuffer.getInt() === BooleanBitSetCompressionType.typeID) + assert(compressedBuffer.getInt() === 64 + 2) + assert(compressedBuffer.getLong() === 682) + assert(compressedBuffer.getLong() === 2) + assert(!compressedBuffer.hasRemaining) + } +} diff --git a/src/main/test/scala/shark/memstore2/column/NullableColumnBuilderSuite.scala b/src/main/test/scala/shark/memstore2/column/NullableColumnBuilderSuite.scala new file mode 100755 index 00000000..2208569e --- /dev/null +++ b/src/main/test/scala/shark/memstore2/column/NullableColumnBuilderSuite.scala @@ -0,0 +1,145 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2.column + +import org.apache.hadoop.io.Text +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory + +import org.scalatest.FunSuite + +class NullableColumnBuilderSuite extends FunSuite { + + test("Empty column") { + val c = new IntColumnBuilder() + c.initialize(4) + val b = c.build() + // # of nulls + assert(b.getInt() === 0) + // column type + assert(b.getInt() === INT.typeID) + assert(b.getInt() === DefaultCompressionType.typeID) + assert(!b.hasRemaining) + } + + test("Buffer size auto growth") { + val c = new StringColumnBuilder() + c.initialize(4) + val oi = PrimitiveObjectInspectorFactory.writableStringObjectInspector + c.append(new Text("a"), oi) + c.append(null, oi) + c.append(new Text("b"), oi) + c.append(null, oi) + c.append(new Text("abc"), oi) + c.append(null, oi) + c.append(null, oi) + c.append(new Text("efg"), oi) + val b = c.build() + b.position(4 + 4 * 4) + val colType = b.getInt() + assert(colType === STRING.typeID) + } + + test("Null Strings") { + val c = new StringColumnBuilder() + c.initialize(4) + val oi = PrimitiveObjectInspectorFactory.writableStringObjectInspector + c.append(new Text("a"), oi) + c.append(null, oi) + c.append(new Text("b"), oi) + c.append(null, oi) + val b = c.build() + + // Number of nulls + assert(b.getInt() === 2) + + // First null position is 1, and then 3 + assert(b.getInt() === 1) + assert(b.getInt() === 3) + + // Column data type + assert(b.getInt() === STRING.typeID) + + // Compression type + assert(b.getInt() === DefaultCompressionType.typeID) + + // Data + assert(b.getInt() === 1) + assert(b.get() === 97) + assert(b.getInt() === 1) + assert(b.get() === 98) + } + + test("Null Ints") { + val c = new IntColumnBuilder() + c.initialize(4) + val oi = PrimitiveObjectInspectorFactory.javaIntObjectInspector + c.append(123.asInstanceOf[Object], oi) + c.append(null, oi) + c.append(null, oi) + c.append(56.asInstanceOf[Object], oi) + val b = c.build() + + // # of nulls and null positions + assert(b.getInt() === 2) + assert(b.getInt() === 1) + assert(b.getInt() === 2) + + // non nulls + assert(b.getInt() === INT.typeID) + assert(b.getInt() === DefaultCompressionType.typeID) + assert(b.getInt() === 123) + } + + test("Nullable Ints 2") { + val c = new IntColumnBuilder() + c.initialize(4) + val oi = PrimitiveObjectInspectorFactory.javaIntObjectInspector + Range(1, 1000).foreach { x => + c.append(x.asInstanceOf[Object], oi) + } + val b = c.build() + // null count + assert(b.getInt() === 0) + // column type + assert(b.getInt() === INT.typeID) + // compression type + assert(b.getInt() === DefaultCompressionType.typeID) + } + + test("Null Longs") { + val c = new LongColumnBuilder() + c.initialize(4) + val oi = PrimitiveObjectInspectorFactory.javaLongObjectInspector + c.append(123L.asInstanceOf[Object], oi) + c.append(null, oi) + c.append(null, oi) + c.append(56L.asInstanceOf[Object], oi) + val b = c.build() + + // # of nulls and null positions + assert(b.getInt() === 2) + assert(b.getInt() === 1) + assert(b.getInt() === 2) + + // non-nulls + assert(b.getInt() === LONG.typeID) + assert(b.getInt() === DefaultCompressionType.typeID) + assert(b.getLong() === 123L) + } + +} diff --git a/src/main/test/scala/shark/memstore2/column/NullableColumnIteratorSuite.scala b/src/main/test/scala/shark/memstore2/column/NullableColumnIteratorSuite.scala new file mode 100755 index 00000000..d4784fa5 --- /dev/null +++ b/src/main/test/scala/shark/memstore2/column/NullableColumnIteratorSuite.scala @@ -0,0 +1,118 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2.column + +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory +import org.apache.hadoop.io.Text +import org.apache.hadoop.io.IntWritable + +import org.scalatest.FunSuite + + +class NullableColumnIteratorSuite extends FunSuite { + + test("String Growth") { + val c = new StringColumnBuilder + c.initialize(4, "") + val oi = PrimitiveObjectInspectorFactory.writableStringObjectInspector + + val a = Array[Text]( + new Text("a"), null, + new Text("b"), null, + new Text("abc"), null, + null, null, new Text("efg") + ) + a.foreach { + t => c.append(t, oi) + } + val b = c.build() + val i = ColumnIterator.newIterator(b) + Range(0, a.length).foreach { x => + if (x > 0) assert(i.hasNext) + i.next() + val v = i.current + if (a(x) == null) { + assert(v == null) + } else { + assert(v.toString == a(x).toString) + } + } + assert(!i.hasNext) + } + + test("Iterate Strings") { + val c = new StringColumnBuilder + c.initialize(4, "") + val oi = PrimitiveObjectInspectorFactory.writableStringObjectInspector + + c.append(new Text("a"), oi) + c.append(new Text(""), oi) + c.append(null, oi) + c.append(new Text("b"), oi) + c.append(new Text("Abcdz"), oi) + c.append(null, oi) + val b = c.build() + val i = ColumnIterator.newIterator(b) + i.next() + assert(i.current.toString() == "a") + i.next() + assert(i.current.toString() == "") + i.next() + assert(i.current == null) + i.next() + assert(i.current.toString() == "b") + i.next() + assert(i.current.toString() == "Abcdz") + i.next() + assert(i.current == null) + assert(false === i.hasNext) + } + + test("Iterate Ints") { + def testList(l: Seq[AnyRef]) { + val c = new IntColumnBuilder + c.initialize(l.size, "") + val oi = PrimitiveObjectInspectorFactory.javaIntObjectInspector + + l.foreach { item => + if (item == null) { + c.append(null, oi) + } else { + c.append(item.asInstanceOf[Object], oi) + } + } + + val b = c.build() + val i = ColumnIterator.newIterator(b) + + l.foreach { x => + i.next() + if (x == null) { + assert(i.current === x) + } else { + assert(i.current.asInstanceOf[IntWritable].get === x) + } + } + assert(false === i.hasNext) + } + + testList(List(null, null, 123.asInstanceOf[AnyRef])) + testList(List(123.asInstanceOf[AnyRef], 4.asInstanceOf[AnyRef], null)) + testList(List(null)) + } +} diff --git a/src/main/test/scala/shark/util/BloomFilterSuite.scala b/src/main/test/scala/shark/util/BloomFilterSuite.scala new file mode 100755 index 00000000..8c1507d0 --- /dev/null +++ b/src/main/test/scala/shark/util/BloomFilterSuite.scala @@ -0,0 +1,35 @@ +package shark.util + +import org.scalatest.FunSuite + +class BloomFilterSuite extends FunSuite{ + + test("Integer") { + val bf = new BloomFilter(0.03, 1000000) + Range(0, 1000000).foreach { + i => bf.add(i) + } + assert(bf.contains(333)) + assert(bf.contains(678)) + assert(!bf.contains(1200000)) + } + + test("Integer FP") { + val bf = new BloomFilter(0.03,1000) + Range(0,700).foreach { + i => bf.add(i) + } + assert(bf.contains(333)) + assert(bf.contains(678)) + //is the fraction of false positives in line with what we expect ? + val e = Range(0, 100).map { + i => bf.contains(i*10) + } + val s = e.groupBy(x => x).map(x => (x._1, x._2.size)) + val t = s(true) + val f = s(false) + assert(f > 25 && f < 35) + assert(t < 75 && t > 65) + // expect false positive to be < 3 % and no false negatives + } +} \ No newline at end of file diff --git a/src/main/test/tests_fail.txt b/src/main/test/tests_fail.txt new file mode 100755 index 00000000..5e66a75c --- /dev/null +++ b/src/main/test/tests_fail.txt @@ -0,0 +1,232 @@ +testCliDriver_allcolref_in_udf +testCliDriver_alter_merge_orc +testCliDriver_alter_merge_orc_stats +testCliDriver_alter_numbuckets_partitioned_table +testCliDriver_alter_numbuckets_partitioned_table2 +testCliDriver_alter_partition_coltype +testCliDriver_alter_table_serde2 +testCliDriver_alter_view_as_select +testCliDriver_archive_multi +testCliDriver_auto_join25 +testCliDriver_auto_join29 +testCliDriver_auto_join32 +testCliDriver_auto_join7 +testCliDriver_auto_join_reordering_values +testCliDriver_auto_smb_mapjoin_14 +testCliDriver_auto_sortmerge_join_1 +testCliDriver_auto_sortmerge_join_10 +testCliDriver_auto_sortmerge_join_2 +testCliDriver_auto_sortmerge_join_3 +testCliDriver_auto_sortmerge_join_4 +testCliDriver_auto_sortmerge_join_5 +testCliDriver_auto_sortmerge_join_6 +testCliDriver_auto_sortmerge_join_7 +testCliDriver_auto_sortmerge_join_8 +testCliDriver_auto_sortmerge_join_9 +testCliDriver_avro_joins +testCliDriver_ba_table_udfs +testCliDriver_bucket2 +testCliDriver_bucket3 +testCliDriver_bucket_groupby +testCliDriver_bucketcontext_1 +testCliDriver_bucketcontext_2 +testCliDriver_bucketcontext_3 +testCliDriver_bucketcontext_4 +testCliDriver_bucketcontext_5 +testCliDriver_bucketcontext_6 +testCliDriver_bucketcontext_7 +testCliDriver_bucketcontext_8 +testCliDriver_bucketizedhiveinputformat_auto +testCliDriver_bucketmapjoin1 +testCliDriver_bucketmapjoin13 +testCliDriver_bucketmapjoin2 +testCliDriver_bucketmapjoin3 +testCliDriver_bucketmapjoin4 +testCliDriver_bucketmapjoin5 +testCliDriver_column_access_stats +testCliDriver_columnstats_partlvl +testCliDriver_columnstats_tbllvl +testCliDriver_combine2_hadoop20 +testCliDriver_combine3 +testCliDriver_concatenate_inherit_table_location +testCliDriver_ctas_hadoop20 +testCliDriver_ctas_uses_database_location +testCliDriver_custom_input_output_format +testCliDriver_escape1 +testCliDriver_escape2 +testCliDriver_exim_04_evolved_parts +testCliDriver_exim_11_managed_external +testCliDriver_exim_12_external_location +testCliDriver_exim_13_managed_location +testCliDriver_exim_14_managed_location_over_existing +testCliDriver_exim_15_external_part +testCliDriver_exim_16_part_external +testCliDriver_exim_17_part_managed +testCliDriver_exim_18_part_external +testCliDriver_exim_19_00_part_external_location +testCliDriver_exim_19_part_external_location +testCliDriver_exim_20_part_managed_location +testCliDriver_filter_join_breaktask +testCliDriver_filter_join_breaktask2 +testCliDriver_groupby10 +testCliDriver_groupby11 +testCliDriver_groupby7_map_multi_single_reducer +testCliDriver_groupby7_noskew_multi_single_reducer +testCliDriver_groupby8 +testCliDriver_groupby8_map +testCliDriver_groupby8_map_skew +testCliDriver_groupby8_noskew +testCliDriver_groupby9 +testCliDriver_groupby_complex_types_multi_single_reducer +testCliDriver_groupby_multi_insert_common_distinct +testCliDriver_groupby_multi_single_reducer +testCliDriver_groupby_multi_single_reducer2 +testCliDriver_groupby_multi_single_reducer3 +testCliDriver_groupby_position +testCliDriver_groupby_sort_1 +testCliDriver_groupby_sort_6 +testCliDriver_groupby_sort_skew_1 +testCliDriver_hook_context_cs +testCliDriver_index_auto_update +testCliDriver_index_bitmap_rc +testCliDriver_infer_bucket_sort +testCliDriver_infer_bucket_sort_convert_join +testCliDriver_infer_bucket_sort_grouping_operators +testCliDriver_infer_bucket_sort_multi_insert +testCliDriver_input3_limit +testCliDriver_input42 +testCliDriver_input_part8 +testCliDriver_join32_lessSize +testCliDriver_join7 +testCliDriver_join_1to1 +testCliDriver_join_filters +testCliDriver_join_filters_overlap +testCliDriver_join_map_ppr +testCliDriver_join_nullsafe +testCliDriver_join_reorder +testCliDriver_join_reorder2 +testCliDriver_join_reorder3 +testCliDriver_join_vc +testCliDriver_leadlag +testCliDriver_leadlag_queries +testCliDriver_mapjoin_hook +testCliDriver_merge3 +testCliDriver_merge4 +testCliDriver_merge_dynamic_partition +testCliDriver_merge_dynamic_partition2 +testCliDriver_merge_dynamic_partition3 +testCliDriver_metadataonly1 +testCliDriver_multi_insert +testCliDriver_multi_insert_gby +testCliDriver_multi_insert_move_tasks_share_dependencies +testCliDriver_nestedvirtual +testCliDriver_nonmr_fetch +testCliDriver_null_column +testCliDriver_orc_create +testCliDriver_orc_empty_files +testCliDriver_overridden_confs +testCliDriver_parallel +testCliDriver_partInit +testCliDriver_partition_wise_fileformat14 +testCliDriver_pcr +testCliDriver_ppd_union_view +testCliDriver_ptf +testCliDriver_ptf_decimal +testCliDriver_ptf_matchpath +testCliDriver_ptf_rcfile +testCliDriver_ptf_register_tblfn +testCliDriver_ptf_seqfile +testCliDriver_query_result_fileformat +testCliDriver_rand_partitionpruner2 +testCliDriver_reduce_deduplicate_extended +testCliDriver_rename_partition_location +testCliDriver_rename_table_location +testCliDriver_sample10 +testCliDriver_sample6 +testCliDriver_select_transform_hint +testCliDriver_semicolon +testCliDriver_semijoin +testCliDriver_show_functions +testCliDriver_skewjoin +testCliDriver_skewjoinopt1 +testCliDriver_skewjoinopt10 +testCliDriver_skewjoinopt11 +testCliDriver_skewjoinopt12 +testCliDriver_skewjoinopt14 +testCliDriver_skewjoinopt15 +testCliDriver_skewjoinopt16 +testCliDriver_skewjoinopt17 +testCliDriver_skewjoinopt19 +testCliDriver_skewjoinopt2 +testCliDriver_skewjoinopt20 +testCliDriver_skewjoinopt3 +testCliDriver_skewjoinopt4 +testCliDriver_skewjoinopt5 +testCliDriver_skewjoinopt6 +testCliDriver_skewjoinopt7 +testCliDriver_skewjoinopt8 +testCliDriver_smb_mapjoin9 +testCliDriver_smb_mapjoin_1 +testCliDriver_smb_mapjoin_11 +testCliDriver_smb_mapjoin_12 +testCliDriver_smb_mapjoin_13 +testCliDriver_smb_mapjoin_14 +testCliDriver_smb_mapjoin_15 +testCliDriver_smb_mapjoin_16 +testCliDriver_smb_mapjoin_17 +testCliDriver_smb_mapjoin_18 +testCliDriver_smb_mapjoin_19 +testCliDriver_smb_mapjoin_2 +testCliDriver_smb_mapjoin_20 +testCliDriver_smb_mapjoin_21 +testCliDriver_smb_mapjoin_22 +testCliDriver_smb_mapjoin_3 +testCliDriver_smb_mapjoin_4 +testCliDriver_smb_mapjoin_5 +testCliDriver_smb_mapjoin_6 +testCliDriver_smb_mapjoin_7 +testCliDriver_sort_merge_join_desc_1 +testCliDriver_sort_merge_join_desc_2 +testCliDriver_sort_merge_join_desc_3 +testCliDriver_sort_merge_join_desc_5 +testCliDriver_sort_merge_join_desc_6 +testCliDriver_sort_merge_join_desc_7 +testCliDriver_stats1 +testCliDriver_stats10 +testCliDriver_stats11 +testCliDriver_stats12 +testCliDriver_stats13 +testCliDriver_stats14 +testCliDriver_stats15 +testCliDriver_stats16 +testCliDriver_stats18 +testCliDriver_stats19 +testCliDriver_stats2 +testCliDriver_stats20 +testCliDriver_stats3 +testCliDriver_stats4 +testCliDriver_stats5 +testCliDriver_stats6 +testCliDriver_stats7 +testCliDriver_stats8 +testCliDriver_stats9 +testCliDriver_stats_empty_partition +testCliDriver_stats_noscan_1 +testCliDriver_stats_noscan_2 +testCliDriver_stats_partscan_1 +testCliDriver_table_access_keys_stats +testCliDriver_udf_bitmap_and +testCliDriver_udf_bitmap_empty +testCliDriver_udf_bitmap_or +testCliDriver_udf_context_aware +testCliDriver_udf_in_file +testCliDriver_union17 +testCliDriver_union22 +testCliDriver_union24 +testCliDriver_union31 +testCliDriver_union_null +testCliDriver_uniquejoin +testCliDriver_virtual_column +testCliDriver_windowing +testCliDriver_windowing_expressions +testCliDriver_windowing_multipartitioning diff --git a/src/main/test/tests_pass.txt b/src/main/test/tests_pass.txt new file mode 100755 index 00000000..389c3779 --- /dev/null +++ b/src/main/test/tests_pass.txt @@ -0,0 +1,925 @@ +testCliDriver_add_part_exist +testCliDriver_add_partition_no_whitelist +testCliDriver_add_partition_with_whitelist +testCliDriver_alias_casted_column +testCliDriver_alter1 +testCliDriver_alter2 +testCliDriver_alter3 +testCliDriver_alter4 +testCliDriver_alter5 +testCliDriver_alter_concatenate_indexed_table +testCliDriver_alter_index +testCliDriver_alter_merge +testCliDriver_alter_merge_2 +testCliDriver_alter_merge_orc2 +testCliDriver_alter_merge_stats +testCliDriver_alter_partition_clusterby_sortby +testCliDriver_alter_partition_format_loc +testCliDriver_alter_partition_protect_mode +testCliDriver_alter_partition_with_whitelist +testCliDriver_alter_rename_partition +testCliDriver_alter_rename_partition_authorization +testCliDriver_alter_skewed_table +testCliDriver_alter_table_not_sorted +testCliDriver_alter_table_serde +testCliDriver_alter_view_rename +testCliDriver_ambiguous_col +testCliDriver_authorization_1 +testCliDriver_authorization_2 +testCliDriver_authorization_3 +testCliDriver_authorization_4 +testCliDriver_authorization_5 +testCliDriver_authorization_6 +testCliDriver_authorization_7 +testCliDriver_autogen_colalias +testCliDriver_auto_join0 +testCliDriver_auto_join1 +testCliDriver_auto_join10 +testCliDriver_auto_join11 +testCliDriver_auto_join12 +testCliDriver_auto_join13 +testCliDriver_auto_join14 +testCliDriver_auto_join14_hadoop20 +testCliDriver_auto_join15 +testCliDriver_auto_join16 +testCliDriver_auto_join17 +testCliDriver_auto_join18 +testCliDriver_auto_join18_multi_distinct +testCliDriver_auto_join19 +testCliDriver_auto_join2 +testCliDriver_auto_join20 +testCliDriver_auto_join21 +testCliDriver_auto_join22 +testCliDriver_auto_join23 +testCliDriver_auto_join24 +testCliDriver_auto_join26 +testCliDriver_auto_join27 +testCliDriver_auto_join28 +testCliDriver_auto_join3 +testCliDriver_auto_join30 +testCliDriver_auto_join31 +testCliDriver_auto_join4 +testCliDriver_auto_join5 +testCliDriver_auto_join6 +testCliDriver_auto_join8 +testCliDriver_auto_join9 +testCliDriver_auto_join_filters +testCliDriver_auto_join_nulls +testCliDriver_auto_sortmerge_join_11 +testCliDriver_auto_sortmerge_join_12 +testCliDriver_avro_change_schema +testCliDriver_avro_compression_enabled +testCliDriver_avro_evolved_schemas +testCliDriver_avro_nullable_fields +testCliDriver_avro_sanity_test +testCliDriver_avro_schema_error_message +testCliDriver_avro_schema_literal +testCliDriver_ba_table1 +testCliDriver_ba_table2 +testCliDriver_ba_table3 +testCliDriver_ba_table_union +testCliDriver_binary_constant +testCliDriver_binary_output_format +testCliDriver_binarysortable_1 +testCliDriver_binary_table_bincolserde +testCliDriver_binary_table_colserde +testCliDriver_bucket1 +testCliDriver_bucket_map_join_1 +testCliDriver_bucketmapjoin10 +testCliDriver_bucketmapjoin11 +testCliDriver_bucketmapjoin12 +testCliDriver_bucket_map_join_2 +testCliDriver_bucketmapjoin8 +testCliDriver_bucketmapjoin9 +testCliDriver_bucketmapjoin_negative +testCliDriver_bucketmapjoin_negative2 +testCliDriver_bucketmapjoin_negative3 +testCliDriver_case_sensitivity +testCliDriver_cast1 +testCliDriver_cluster +testCliDriver_columnarserde_create_shortcut +testCliDriver_combine1 +testCliDriver_compute_stats_binary +testCliDriver_compute_stats_boolean +testCliDriver_compute_stats_double +testCliDriver_compute_stats_empty_table +testCliDriver_compute_stats_long +testCliDriver_compute_stats_string +testCliDriver_constant_prop +testCliDriver_convert_enum_to_string +testCliDriver_count +testCliDriver_cp_mj_rc +testCliDriver_create_1 +testCliDriver_create_alter_list_bucketing_table1 +testCliDriver_create_big_view +testCliDriver_create_default_prop +testCliDriver_create_escape +testCliDriver_create_genericudaf +testCliDriver_create_genericudf +testCliDriver_create_insert_outputformat +testCliDriver_create_like +testCliDriver_create_like2 +testCliDriver_create_like_tbl_props +testCliDriver_create_like_view +testCliDriver_create_merge_compressed +testCliDriver_create_nested_type +testCliDriver_create_or_replace_view +testCliDriver_create_skewed_table1 +testCliDriver_create_struct_table +testCliDriver_create_udaf +testCliDriver_create_union_table +testCliDriver_create_view +testCliDriver_create_view_partitioned +testCliDriver_cross_join +testCliDriver_ct_case_insensitive +testCliDriver_database +testCliDriver_database_drop +testCliDriver_database_location +testCliDriver_database_properties +testCliDriver_ddltime +testCliDriver_decimal_1 +testCliDriver_decimal_2 +testCliDriver_decimal_3 +testCliDriver_decimal_4 +testCliDriver_decimal_precision +testCliDriver_decimal_serde +testCliDriver_decimal_udf +testCliDriver_default_partition_name +testCliDriver_delimiter +testCliDriver_desc_non_existent_tbl +testCliDriver_describe_comment_indent +testCliDriver_describe_database_json +testCliDriver_describe_formatted_view_partitioned +testCliDriver_describe_formatted_view_partitioned_json +testCliDriver_describe_pretty +testCliDriver_describe_syntax +testCliDriver_describe_table +testCliDriver_describe_table_json +testCliDriver_describe_xpath +testCliDriver_diff_part_input_formats +testCliDriver_disable_file_format_check +testCliDriver_driverhook +testCliDriver_drop_database_removes_partition_dirs +testCliDriver_drop_function +testCliDriver_drop_index +testCliDriver_drop_index_removes_partition_dirs +testCliDriver_drop_multi_partitions +testCliDriver_drop_partitions_filter +testCliDriver_drop_partitions_filter2 +testCliDriver_drop_partitions_filter3 +testCliDriver_drop_partitions_ignore_protection +testCliDriver_drop_table +testCliDriver_drop_table2 +testCliDriver_drop_table_removes_partition_dirs +testCliDriver_drop_udf +testCliDriver_drop_view +testCliDriver_enforce_order +testCliDriver_escape_clusterby1 +testCliDriver_escape_distributeby1 +testCliDriver_escape_orderby1 +testCliDriver_escape_sortby1 +testCliDriver_exim_00_nonpart_empty +testCliDriver_exim_01_nonpart +testCliDriver_exim_02_00_part_empty +testCliDriver_exim_02_part +testCliDriver_exim_03_nonpart_over_compat +testCliDriver_exim_04_all_part +testCliDriver_exim_05_some_part +testCliDriver_exim_06_one_part +testCliDriver_exim_07_all_part_over_nonoverlap +testCliDriver_exim_08_nonpart_rename +testCliDriver_exim_09_part_spec_nonoverlap +testCliDriver_exim_10_external_managed +testCliDriver_exim_21_export_authsuccess +testCliDriver_exim_22_import_exist_authsuccess +testCliDriver_exim_23_import_part_authsuccess +testCliDriver_exim_24_import_nonexist_authsuccess +testCliDriver_explain_dependency +testCliDriver_explain_dependency2 +testCliDriver_explode_null +testCliDriver_fileformat_mix +testCliDriver_fileformat_sequencefile +testCliDriver_fileformat_text +testCliDriver_global_limit +testCliDriver_groupby1 +testCliDriver_groupby1_limit +testCliDriver_groupby1_map +testCliDriver_groupby1_map_nomap +testCliDriver_groupby1_map_skew +testCliDriver_groupby1_noskew +testCliDriver_groupby2_limit +testCliDriver_groupby2_map +testCliDriver_groupby2_map_multi_distinct +testCliDriver_groupby2_map_skew +testCliDriver_groupby2_map_skew_multi_distinct +testCliDriver_groupby2_multi_distinct +testCliDriver_groupby2_noskew +testCliDriver_groupby2_noskew_multi_distinct +testCliDriver_groupby3 +testCliDriver_groupby3_map +testCliDriver_groupby3_map_multi_distinct +testCliDriver_groupby3_map_skew +testCliDriver_groupby3_noskew +testCliDriver_groupby3_noskew_multi_distinct +testCliDriver_groupby4 +testCliDriver_groupby4_map +testCliDriver_groupby4_map_skew +testCliDriver_groupby4_noskew +testCliDriver_groupby5 +testCliDriver_groupby5_map +testCliDriver_groupby5_map_skew +testCliDriver_groupby5_noskew +testCliDriver_groupby6 +testCliDriver_groupby6_map +testCliDriver_groupby6_map_skew +testCliDriver_groupby6_noskew +testCliDriver_groupby7 +testCliDriver_groupby7_map +testCliDriver_groupby7_map_skew +testCliDriver_groupby7_noskew +testCliDriver_groupby_bigdata +testCliDriver_groupby_complex_types +testCliDriver_groupby_cube1 +testCliDriver_groupby_distinct_samekey +testCliDriver_groupby_grouping_id1 +testCliDriver_groupby_grouping_id2 +testCliDriver_groupby_grouping_sets1 +testCliDriver_groupby_grouping_sets2 +testCliDriver_groupby_grouping_sets3 +testCliDriver_groupby_grouping_sets4 +testCliDriver_groupby_grouping_sets5 +testCliDriver_groupby_map_ppr +testCliDriver_groupby_map_ppr_multi_distinct +testCliDriver_groupby_mutli_insert_common_distinct +testCliDriver_groupby_neg_float +testCliDriver_groupby_ppd +testCliDriver_groupby_ppr +testCliDriver_groupby_ppr_multi_distinct +testCliDriver_groupby_rollup1 +testCliDriver_groupby_sort_10 +testCliDriver_groupby_sort_2 +testCliDriver_groupby_sort_3 +testCliDriver_groupby_sort_4 +testCliDriver_groupby_sort_5 +testCliDriver_groupby_sort_7 +testCliDriver_groupby_sort_8 +testCliDriver_groupby_sort_9 +testCliDriver_groupby_sort_test_1 +testCliDriver_having +testCliDriver_hook_order +testCliDriver_implicit_cast1 +testCliDriver_index_auth +testCliDriver_index_auto +testCliDriver_index_auto_empty +testCliDriver_index_auto_file_format +testCliDriver_index_auto_multiple +testCliDriver_index_auto_mult_tables +testCliDriver_index_auto_mult_tables_compact +testCliDriver_index_auto_partitioned +testCliDriver_index_auto_self_join +testCliDriver_index_auto_unused +testCliDriver_index_bitmap +testCliDriver_index_bitmap1 +testCliDriver_index_bitmap2 +testCliDriver_index_bitmap3 +testCliDriver_index_bitmap_auto +testCliDriver_index_bitmap_auto_partitioned +testCliDriver_index_bitmap_compression +testCliDriver_index_compact +testCliDriver_index_compact_1 +testCliDriver_index_compact_2 +testCliDriver_index_compact_3 +testCliDriver_index_compact_binary_search +testCliDriver_index_compression +testCliDriver_index_creation +testCliDriver_index_stale +testCliDriver_index_stale_partitioned +testCliDriver_infer_const_type +testCliDriver_init_file +testCliDriver_innerjoin +testCliDriver_inoutdriver +testCliDriver_input +testCliDriver_input0 +testCliDriver_input1 +testCliDriver_input10 +testCliDriver_input11 +testCliDriver_input11_limit +testCliDriver_input12_hadoop20 +testCliDriver_input13 +testCliDriver_input14 +testCliDriver_input14_limit +testCliDriver_input15 +testCliDriver_input16 +testCliDriver_input17 +testCliDriver_input18 +testCliDriver_input19 +testCliDriver_input1_limit +testCliDriver_input2 +testCliDriver_input20 +testCliDriver_input21 +testCliDriver_input22 +testCliDriver_input23 +testCliDriver_input24 +testCliDriver_input25 +testCliDriver_input26 +testCliDriver_input28 +testCliDriver_input2_limit +testCliDriver_input3 +testCliDriver_input30 +testCliDriver_input31 +testCliDriver_input32 +testCliDriver_input33 +testCliDriver_input34 +testCliDriver_input35 +testCliDriver_input36 +testCliDriver_input37 +testCliDriver_input38 +testCliDriver_input39_hadoop20 +testCliDriver_input4 +testCliDriver_input40 +testCliDriver_input41 +testCliDriver_input43 +testCliDriver_input44 +testCliDriver_input45 +testCliDriver_input46 +testCliDriver_input49 +testCliDriver_input4_cb_delim +testCliDriver_input4_limit +testCliDriver_input5 +testCliDriver_input6 +testCliDriver_input7 +testCliDriver_input8 +testCliDriver_input9 +testCliDriver_input_columnarserde +testCliDriver_inputddl1 +testCliDriver_inputddl2 +testCliDriver_inputddl3 +testCliDriver_inputddl4 +testCliDriver_inputddl5 +testCliDriver_inputddl6 +testCliDriver_inputddl7 +testCliDriver_inputddl8 +testCliDriver_input_dfs +testCliDriver_input_dynamicserde +testCliDriver_input_lazyserde +testCliDriver_input_limit +testCliDriver_input_part0 +testCliDriver_input_part1 +testCliDriver_input_part10 +testCliDriver_input_part2 +testCliDriver_input_part3 +testCliDriver_input_part4 +testCliDriver_input_part5 +testCliDriver_input_part6 +testCliDriver_input_part7 +testCliDriver_input_part9 +testCliDriver_input_testsequencefile +testCliDriver_input_testxpath +testCliDriver_input_testxpath2 +testCliDriver_input_testxpath3 +testCliDriver_input_testxpath4 +testCliDriver_insert1 +testCliDriver_insert1_overwrite_partitions +testCliDriver_insert2_overwrite_partitions +testCliDriver_insert_compressed +testCliDriver_insertexternal1 +testCliDriver_insert_into1 +testCliDriver_insert_into2 +testCliDriver_insert_into3 +testCliDriver_insert_into4 +testCliDriver_insert_into5 +testCliDriver_insert_into6 +testCliDriver_insert_overwrite_local_directory_1 +testCliDriver_join0 +testCliDriver_join10 +testCliDriver_join11 +testCliDriver_join12 +testCliDriver_join13 +testCliDriver_join14 +testCliDriver_join14_hadoop20 +testCliDriver_join15 +testCliDriver_join16 +testCliDriver_join17 +testCliDriver_join18 +testCliDriver_join18_multi_distinct +testCliDriver_join19 +testCliDriver_join2 +testCliDriver_join20 +testCliDriver_join21 +testCliDriver_join22 +testCliDriver_join23 +testCliDriver_join24 +testCliDriver_join25 +testCliDriver_join26 +testCliDriver_join27 +testCliDriver_join28 +testCliDriver_join29 +testCliDriver_join3 +testCliDriver_join30 +testCliDriver_join31 +testCliDriver_join32 +testCliDriver_join33 +testCliDriver_join34 +testCliDriver_join35 +testCliDriver_join36 +testCliDriver_join37 +testCliDriver_join38 +testCliDriver_join39 +testCliDriver_join4 +testCliDriver_join40 +testCliDriver_join41 +testCliDriver_join5 +testCliDriver_join6 +testCliDriver_join8 +testCliDriver_join9 +testCliDriver_join_array +testCliDriver_join_casesensitive +testCliDriver_join_empty +testCliDriver_join_hive_626 +testCliDriver_join_nulls +testCliDriver_join_rc +testCliDriver_join_reorder4 +testCliDriver_join_star +testCliDriver_join_thrift +testCliDriver_join_view +testCliDriver_keyword_1 +testCliDriver_lateral_view +testCliDriver_lateral_view_cp +testCliDriver_lateral_view_ppd +testCliDriver_leftsemijoin +testCliDriver_lineage1 +testCliDriver_literal_decimal +testCliDriver_literal_double +testCliDriver_literal_ints +testCliDriver_literal_string +testCliDriver_load_binary_data +testCliDriver_load_dyn_part1 +testCliDriver_load_dyn_part10 +testCliDriver_load_dyn_part11 +testCliDriver_load_dyn_part12 +testCliDriver_load_dyn_part13 +testCliDriver_load_dyn_part14 +testCliDriver_load_dyn_part15 +testCliDriver_load_dyn_part2 +testCliDriver_load_dyn_part3 +testCliDriver_load_dyn_part4 +testCliDriver_load_dyn_part5 +testCliDriver_load_dyn_part6 +testCliDriver_load_dyn_part7 +testCliDriver_load_dyn_part8 +testCliDriver_load_dyn_part9 +testCliDriver_load_exist_part_authsuccess +testCliDriver_load_fs +testCliDriver_load_nonpart_authsuccess +testCliDriver_load_overwrite +testCliDriver_loadpart1 +testCliDriver_load_part_authsuccess +testCliDriver_lock1 +testCliDriver_lock2 +testCliDriver_lock3 +testCliDriver_lock4 +testCliDriver_louter_join_ppr +testCliDriver_mapjoin1 +testCliDriver_mapjoin_distinct +testCliDriver_mapjoin_filter_on_outerjoin +testCliDriver_mapjoin_mapjoin +testCliDriver_mapjoin_subquery +testCliDriver_mapjoin_subquery2 +testCliDriver_mapjoin_test_outer +testCliDriver_mapreduce1 +testCliDriver_mapreduce2 +testCliDriver_mapreduce3 +testCliDriver_mapreduce4 +testCliDriver_mapreduce5 +testCliDriver_mapreduce6 +testCliDriver_mapreduce7 +testCliDriver_mapreduce8 +testCliDriver_merge1 +testCliDriver_merge2 +testCliDriver_merge_dynamic_partition4 +testCliDriver_merge_dynamic_partition5 +testCliDriver_mergejoins +testCliDriver_mergejoins_mixed +testCliDriver_metadata_export_drop +testCliDriver_mi +testCliDriver_misc_json +testCliDriver_multigroupby_singlemr +testCliDriver_multi_join_union +testCliDriver_multiMapJoin1 +testCliDriver_multi_sahooks +testCliDriver_newline +testCliDriver_noalias_subq1 +testCliDriver_no_hooks +testCliDriver_nomore_ambiguous_table_col +testCliDriver_nonblock_op_deduplicate +testCliDriver_nonreserved_keywords_input37 +testCliDriver_nonreserved_keywords_insert_into1 +testCliDriver_notable_alias1 +testCliDriver_notable_alias2 +testCliDriver_null_cast +testCliDriver_nullgroup +testCliDriver_nullgroup2 +testCliDriver_nullgroup3 +testCliDriver_nullgroup4 +testCliDriver_nullgroup4_multi_distinct +testCliDriver_nullgroup5 +testCliDriver_nullinput +testCliDriver_nullinput2 +testCliDriver_nullscript +testCliDriver_num_op_type_conv +testCliDriver_ops_comparison +testCliDriver_orc_createas1 +testCliDriver_orc_diff_part_cols +testCliDriver_orc_empty_strings +testCliDriver_orc_ends_with_nulls +testCliDriver_order +testCliDriver_order2 +testCliDriver_outer_join_ppr +testCliDriver_parenthesis_star_by +testCliDriver_partcols1 +testCliDriver_part_inherit_tbl_props +testCliDriver_part_inherit_tbl_props_empty +testCliDriver_part_inherit_tbl_props_with_star +testCliDriver_partition_decode_name +testCliDriver_partition_schema1 +testCliDriver_partition_serde_format +testCliDriver_partitions_json +testCliDriver_partition_special_char +testCliDriver_partition_vs_table_metadata +testCliDriver_partition_wise_fileformat +testCliDriver_partition_wise_fileformat10 +testCliDriver_partition_wise_fileformat11 +testCliDriver_partition_wise_fileformat12 +testCliDriver_partition_wise_fileformat13 +testCliDriver_partition_wise_fileformat15 +testCliDriver_partition_wise_fileformat16 +testCliDriver_partition_wise_fileformat2 +testCliDriver_partition_wise_fileformat3 +testCliDriver_partition_wise_fileformat4 +testCliDriver_partition_wise_fileformat5 +testCliDriver_partition_wise_fileformat6 +testCliDriver_partition_wise_fileformat7 +testCliDriver_partition_wise_fileformat8 +testCliDriver_partition_wise_fileformat9 +testCliDriver_plan_json +testCliDriver_ppd1 +testCliDriver_ppd2 +testCliDriver_ppd_clusterby +testCliDriver_ppd_constant_expr +testCliDriver_ppd_constant_where +testCliDriver_ppd_gby +testCliDriver_ppd_gby2 +testCliDriver_ppd_gby_join +testCliDriver_ppd_join +testCliDriver_ppd_join2 +testCliDriver_ppd_join3 +testCliDriver_ppd_join_filter +testCliDriver_ppd_multi_insert +testCliDriver_ppd_outer_join1 +testCliDriver_ppd_outer_join2 +testCliDriver_ppd_outer_join3 +testCliDriver_ppd_outer_join4 +testCliDriver_ppd_outer_join5 +testCliDriver_ppd_random +testCliDriver_ppd_repeated_alias +testCliDriver_ppd_transform +testCliDriver_ppd_udf_case +testCliDriver_ppd_udf_col +testCliDriver_ppd_union +testCliDriver_ppr_allchildsarenull +testCliDriver_ppr_pushdown +testCliDriver_ppr_pushdown2 +testCliDriver_ppr_pushdown3 +testCliDriver_print_header +testCliDriver_progress_1 +testCliDriver_protectmode +testCliDriver_protectmode2 +testCliDriver_ptf_general_queries +testCliDriver_ql_rewrite_gbtoidx +testCliDriver_query_properties +testCliDriver_query_with_semi +testCliDriver_quote1 +testCliDriver_quote2 +testCliDriver_rand_partitionpruner1 +testCliDriver_rand_partitionpruner3 +testCliDriver_rcfile_bigdata +testCliDriver_rcfile_columnar +testCliDriver_rcfile_createas1 +testCliDriver_rcfile_default_format +testCliDriver_rcfile_lazydecompress +testCliDriver_rcfile_null_value +testCliDriver_rcfile_toleratecorruptions +testCliDriver_rcfile_union +testCliDriver_reduce_deduplicate_exclude_gby +testCliDriver_reduce_deduplicate_exclude_join +testCliDriver_reducesink_dedup +testCliDriver_regex_col +testCliDriver_regexp_extract +testCliDriver_rename_column +testCliDriver_rename_external_partition_location +testCliDriver_repair +testCliDriver_reset_conf +testCliDriver_router_join_ppr +testCliDriver_sample1 +testCliDriver_sample2 +testCliDriver_sample3 +testCliDriver_sample4 +testCliDriver_sample5 +testCliDriver_sample7 +testCliDriver_sample8 +testCliDriver_sample9 +testCliDriver_sample_islocalmode_hook_hadoop20 +testCliDriver_script_env_var1 +testCliDriver_script_env_var2 +testCliDriver_script_pipe +testCliDriver_select_as_omitted +testCliDriver_select_unquote_and +testCliDriver_select_unquote_not +testCliDriver_select_unquote_or +testCliDriver_serde_regex +testCliDriver_serde_reported_schema +testCliDriver_set_processor_namespaces +testCliDriver_set_variable_sub +testCliDriver_show_columns +testCliDriver_show_create_table_alter +testCliDriver_show_create_table_db_table +testCliDriver_show_create_table_delimited +testCliDriver_show_create_table_partitioned +testCliDriver_show_create_table_serde +testCliDriver_show_create_table_view +testCliDriver_show_describe_func_quotes +testCliDriver_show_indexes_edge_cases +testCliDriver_show_indexes_syntax +testCliDriver_show_partitions +testCliDriver_showparts +testCliDriver_show_tables +testCliDriver_show_tablestatus +testCliDriver_show_tblproperties +testCliDriver_shutdown +testCliDriver_skewjoinopt13 +testCliDriver_skewjoinopt18 +testCliDriver_skewjoinopt9 +testCliDriver_skewjoin_union_remove_1 +testCliDriver_skewjoin_union_remove_2 +testCliDriver_smb_mapjoin_10 +testCliDriver_sort +testCliDriver_sort_merge_join_desc_4 +testCliDriver_source +testCliDriver_stats0 +testCliDriver_stats_aggregator_error_1 +testCliDriver_stats_empty_dyn_part +testCliDriver_stats_publisher_error_1 +testCliDriver_str_to_map +testCliDriver_subq +testCliDriver_subq2 +testCliDriver_symlink_text_input_format +testCliDriver_tablename_with_select +testCliDriver_test_boolean_whereclause +testCliDriver_timestamp_1 +testCliDriver_timestamp_2 +testCliDriver_timestamp_3 +testCliDriver_timestamp_comparison +testCliDriver_timestamp_lazy +testCliDriver_timestamp_null +testCliDriver_timestamp_udf +testCliDriver_touch +testCliDriver_transform1 +testCliDriver_transform2 +testCliDriver_transform_ppr1 +testCliDriver_transform_ppr2 +testCliDriver_truncate_table +testCliDriver_type_cast_1 +testCliDriver_type_widening +testCliDriver_udaf_collect_set +testCliDriver_udaf_context_ngrams +testCliDriver_udaf_corr +testCliDriver_udaf_covar_pop +testCliDriver_udaf_covar_samp +testCliDriver_udaf_histogram_numeric +testCliDriver_udaf_ngrams +testCliDriver_udaf_number_format +testCliDriver_udaf_percentile_approx +testCliDriver_udf1 +testCliDriver_udf_10_trims +testCliDriver_udf2 +testCliDriver_udf3 +testCliDriver_udf4 +testCliDriver_udf5 +testCliDriver_udf6 +testCliDriver_udf7 +testCliDriver_udf8 +testCliDriver_udf9 +testCliDriver_udf_abs +testCliDriver_udf_acos +testCliDriver_udf_add +testCliDriver_udf_array +testCliDriver_udf_array_contains +testCliDriver_udf_ascii +testCliDriver_udf_asin +testCliDriver_udf_atan +testCliDriver_udf_avg +testCliDriver_udf_between +testCliDriver_udf_bigint +testCliDriver_udf_bin +testCliDriver_udf_bitwise_and +testCliDriver_udf_bitwise_not +testCliDriver_udf_bitwise_or +testCliDriver_udf_bitwise_xor +testCliDriver_udf_boolean +testCliDriver_udf_case +testCliDriver_udf_case_column_pruning +testCliDriver_udf_case_thrift +testCliDriver_udf_ceil +testCliDriver_udf_ceiling +testCliDriver_udf_coalesce +testCliDriver_udf_compare_java_string +testCliDriver_udf_concat +testCliDriver_udf_concat_insert1 +testCliDriver_udf_concat_insert2 +testCliDriver_udf_concat_ws +testCliDriver_udf_conv +testCliDriver_udf_cos +testCliDriver_udf_count +testCliDriver_udf_date_add +testCliDriver_udf_datediff +testCliDriver_udf_date_sub +testCliDriver_udf_day +testCliDriver_udf_dayofmonth +testCliDriver_udf_degrees +testCliDriver_udf_div +testCliDriver_udf_divide +testCliDriver_udf_double +testCliDriver_udf_E +testCliDriver_udf_elt +testCliDriver_udf_equal +testCliDriver_udf_exp +testCliDriver_udf_explode +testCliDriver_udf_field +testCliDriver_udf_find_in_set +testCliDriver_udf_float +testCliDriver_udf_floor +testCliDriver_udf_format_number +testCliDriver_udf_from_unixtime +testCliDriver_udf_get_json_object +testCliDriver_udf_greaterthan +testCliDriver_udf_greaterthanorequal +testCliDriver_udf_hash +testCliDriver_udf_hex +testCliDriver_udf_hour +testCliDriver_udf_if +testCliDriver_udf_in +testCliDriver_udf_index +testCliDriver_udf_inline +testCliDriver_udf_instr +testCliDriver_udf_int +testCliDriver_udf_isnotnull +testCliDriver_udf_isnull +testCliDriver_udf_isnull_isnotnull +testCliDriver_udf_java_method +testCliDriver_udf_lcase +testCliDriver_udf_length +testCliDriver_udf_lessthan +testCliDriver_udf_lessthanorequal +testCliDriver_udf_like +testCliDriver_udf_ln +testCliDriver_udf_locate +testCliDriver_udf_log +testCliDriver_udf_log10 +testCliDriver_udf_log2 +testCliDriver_udf_logic_java_boolean +testCliDriver_udf_lower +testCliDriver_udf_lpad +testCliDriver_udf_ltrim +testCliDriver_udf_map +testCliDriver_udf_map_keys +testCliDriver_udf_map_values +testCliDriver_udf_max +testCliDriver_udf_min +testCliDriver_udf_minute +testCliDriver_udf_modulo +testCliDriver_udf_month +testCliDriver_udf_named_struct +testCliDriver_udf_negative +testCliDriver_udf_not +testCliDriver_udf_notequal +testCliDriver_udf_notop +testCliDriver_udf_nvl +testCliDriver_udf_or +testCliDriver_udf_parse_url +testCliDriver_udf_percentile +testCliDriver_udf_PI +testCliDriver_udf_pmod +testCliDriver_udf_positive +testCliDriver_udf_pow +testCliDriver_udf_power +testCliDriver_udf_printf +testCliDriver_udf_radians +testCliDriver_udf_rand +testCliDriver_udf_reflect +testCliDriver_udf_reflect2 +testCliDriver_udf_regexp +testCliDriver_udf_regexp_extract +testCliDriver_udf_regexp_replace +testCliDriver_udf_repeat +testCliDriver_udf_reverse +testCliDriver_udf_rlike +testCliDriver_udf_round +testCliDriver_udf_round_2 +testCliDriver_udf_round_3 +testCliDriver_udf_rpad +testCliDriver_udf_rtrim +testCliDriver_udf_second +testCliDriver_udf_sentences +testCliDriver_udf_sign +testCliDriver_udf_sin +testCliDriver_udf_size +testCliDriver_udf_smallint +testCliDriver_udf_sort_array +testCliDriver_udf_space +testCliDriver_udf_split +testCliDriver_udf_sqrt +testCliDriver_udf_std +testCliDriver_udf_stddev +testCliDriver_udf_stddev_pop +testCliDriver_udf_stddev_samp +testCliDriver_udf_string +testCliDriver_udf_struct +testCliDriver_udf_substr +testCliDriver_udf_substring +testCliDriver_udf_subtract +testCliDriver_udf_sum +testCliDriver_udf_tan +testCliDriver_udf_testlength +testCliDriver_udf_testlength2 +testCliDriver_udf_tinyint +testCliDriver_udf_to_date +testCliDriver_udf_to_unix_timestamp +testCliDriver_udf_translate +testCliDriver_udf_trim +testCliDriver_udf_ucase +testCliDriver_udf_unhex +testCliDriver_udf_union +testCliDriver_udf_unix_timestamp +testCliDriver_udf_upper +testCliDriver_udf_variance +testCliDriver_udf_var_pop +testCliDriver_udf_var_samp +testCliDriver_udf_weekofyear +testCliDriver_udf_when +testCliDriver_udf_xpath +testCliDriver_udf_xpath_boolean +testCliDriver_udf_xpath_double +testCliDriver_udf_xpath_float +testCliDriver_udf_xpath_int +testCliDriver_udf_xpath_long +testCliDriver_udf_xpath_short +testCliDriver_udf_xpath_string +testCliDriver_udtf_explode +testCliDriver_udtf_json_tuple +testCliDriver_udtf_parse_url_tuple +testCliDriver_udtf_stack +testCliDriver_union +testCliDriver_union10 +testCliDriver_union11 +testCliDriver_union12 +testCliDriver_union13 +testCliDriver_union14 +testCliDriver_union15 +testCliDriver_union16 +testCliDriver_union18 +testCliDriver_union19 +testCliDriver_union2 +testCliDriver_union20 +testCliDriver_union21 +testCliDriver_union23 +testCliDriver_union25 +testCliDriver_union26 +testCliDriver_union27 +testCliDriver_union28 +testCliDriver_union29 +testCliDriver_union3 +testCliDriver_union30 +testCliDriver_union32 +testCliDriver_union33 +testCliDriver_union34 +testCliDriver_union4 +testCliDriver_union5 +testCliDriver_union6 +testCliDriver_union7 +testCliDriver_union8 +testCliDriver_union9 +testCliDriver_union_lateralview +testCliDriver_union_ppr +testCliDriver_union_remove_6 +testCliDriver_union_script +testCliDriver_union_view +testCliDriver_unset_table_view_property +testCliDriver_updateAccessTime +testCliDriver_view +testCliDriver_view_inputs +testCliDriver_windowing_columnPruning +testCliDriver_windowing_navfn +testCliDriver_windowing_ntile +testCliDriver_windowing_rank +testCliDriver_windowing_udaf +testCliDriver_windowing_windowspec From 5a3d9f826ac8efadb7e57ffc4679e34cacdd0fe6 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 3 Jun 2014 16:12:30 +0800 Subject: [PATCH 02/21] update the jar dependencies --- project/SharkBuild.scala | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index 0ba800b1..3aff0e25 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -77,10 +77,9 @@ object SharkBuild extends Build { /** Extra artifacts not included in Spark SQL's Hive support. */ - val hiveArtifacts = Seq("hive-cli", "hive-jdbc") + val hiveArtifacts = Seq("hive-cli", "hive-jdbc", "hive-exec", "hive-service") val hiveDependencies = hiveArtifacts.map ( artifactId => - "org.spark-project.hive" % artifactId % "0.12.0" excludeAll( - excludeGuava, excludeLog4j, excludeAsm, excludeNetty, excludeXerces, excludeServlet) + "org.spark-project.hive" % artifactId % "0.12.0" ) val yarnDependency = (if (YARN_ENABLED) { @@ -110,9 +109,9 @@ object SharkBuild extends Build { // Download managed jars into lib_managed. retrieveManaged := true, resolvers ++= Seq( + "Local Maven" at Path.userHome.asFile.toURI.toURL + ".m2/repository", "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/", - "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/", - "Local Maven" at Path.userHome.asFile.toURI.toURL + ".m2/repository" + "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/" ), publishTo <<= version { (v: String) => From 0c2d7f6bfcd876e6068fdbb385b30c62e6c922b5 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 3 Jun 2014 23:17:49 +0800 Subject: [PATCH 03/21] Fix ClassCastException --- project/SharkBuild.scala | 8 ++++ src/main/scala/shark/CatalystContext.scala | 49 +--------------------- src/main/scala/shark/CatalystDriver.scala | 1 + src/main/scala/shark/SharkCliDriver.scala | 21 ++++++---- 4 files changed, 22 insertions(+), 57 deletions(-) diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index 3aff0e25..391c80ff 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -38,6 +38,7 @@ object SharkBuild extends Build { val SCALAC_JVM_VERSION = "jvm-1.6" val JAVAC_JVM_VERSION = "1.6" + val JETTY_VERSION = "8.1.14.v20131031" // Hadoop version to build against. For example, "0.20.2", "0.20.205.0", or // "1.0.1" for Apache releases, or "0.20.2-cdh3u3" for Cloudera Hadoop. @@ -100,6 +101,10 @@ object SharkBuild extends Build { libraryDependencies ++= hiveDependencies ++ yarnDependency, libraryDependencies ++= Seq( + "io.netty" % "netty-all" % "4.0.17.Final", + "org.eclipse.jetty" % "jetty-server" % JETTY_VERSION, + "org.eclipse.jetty" % "jetty-util" % JETTY_VERSION, + "javax.servlet" % "javax.servlet-api" % "3.0.1", "org.apache.spark" %% "spark-hive" % SPARK_VERSION, "org.apache.spark" %% "spark-repl" % SPARK_VERSION, "com.typesafe" %% "scalalogging-slf4j" % "1.0.1", @@ -110,6 +115,9 @@ object SharkBuild extends Build { retrieveManaged := true, resolvers ++= Seq( "Local Maven" at Path.userHome.asFile.toURI.toURL + ".m2/repository", + "Maven Repository" at "http://repo.maven.apache.org/maven2", + "Apache Repository" at "https://repository.apache.org/content/repositories/releases", + "JBoss Repository" at "https://repository.jboss.org/nexus/content/repositories/releases/", "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/", "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/" ), diff --git a/src/main/scala/shark/CatalystContext.scala b/src/main/scala/shark/CatalystContext.scala index 6f455d8e..d636e598 100644 --- a/src/main/scala/shark/CatalystContext.scala +++ b/src/main/scala/shark/CatalystContext.scala @@ -86,8 +86,6 @@ case class CatalystContext(sc: SparkContext) extends HiveContext(sc) with LogHel val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() val proc: CommandProcessor = CommandProcessorFactory.get(tokens(0), hiveconf) - SessionState.start(sessionState) - proc match { case driver: Driver => driver.init() @@ -104,7 +102,7 @@ case class CatalystContext(sc: SparkContext) extends HiveContext(sc) with LogHel driver.destroy() (0, results) case _ => - sessionState.out.println(tokens(0) + " " + cmd_1) + SessionState.get().out.println(tokens(0) + " " + cmd_1) (proc.run(cmd_1).getResponseCode, Seq[String]()) } } catch { @@ -122,49 +120,4 @@ case class CatalystContext(sc: SparkContext) extends HiveContext(sc) with LogHel throw e } } - - override lazy val hiveconf = new HiveConf(classOf[SessionState]) - override lazy val sessionState = new SessionState(hiveconf) -} - -abstract class Launcher(cc: CatalystContext) { - def execute(hql: String): (Int, Seq[String]) = new cc.HiveQLQueryExecution(hql).result() -} - -private[hive] case class HiveLauncher(cc: CatalystContext) extends Launcher(cc) { - override def execute(hql: String): (Int, Seq[String]) = cc.runOnHive(hql) -} - -private[hive] case class SparkLauncher(cc: CatalystContext) extends Launcher(cc) - -object CatalystContextWrapper { - val EXEC_MODE = "catalyst.exec.mode" - val EXEC_MODE_SPARK = "spark" - val EXEC_MODE_HIVE = "hive" } - -class CatalystContextWrapper(cc: CatalystContext) { - val candidates = (CatalystContextWrapper.EXEC_MODE_SPARK, SparkLauncher(cc)) :: - (CatalystContextWrapper.EXEC_MODE_HIVE, HiveLauncher(cc)) :: Nil - -// // Use reflection to get access to the two fields. -// val getFormattedDbMethod = classOf[CliDriver].getDeclaredMethod( -// "getFormattedDb", classOf[HiveConf], classOf[CliSessionState]) -// getFormattedDbMethod.setAccessible(true) -// -// val spacesForStringMethod = classOf[CliDriver].getDeclaredMethod( -// "spacesForString", classOf[String]) -// spacesForStringMethod.setAccessible(true) - - def env: (String, org.apache.spark.sql.hive.Launcher) = { - val conf: HiveConf = cc.sessionState.getConf() -// val db = getFormattedDbMethod.invoke(null, conf, ss).asInstanceOf[String] - val cli = conf.get(CatalystContextWrapper.EXEC_MODE, CatalystContextWrapper.EXEC_MODE_SPARK) - - var launcher = candidates.find(_._1 == cli).getOrElse(candidates.head) - -// val promptStr = if (db != null) launcher._1 + db else "" - - (launcher._1, launcher._2) - } -} \ No newline at end of file diff --git a/src/main/scala/shark/CatalystDriver.scala b/src/main/scala/shark/CatalystDriver.scala index 25faa431..bb731474 100644 --- a/src/main/scala/shark/CatalystDriver.scala +++ b/src/main/scala/shark/CatalystDriver.scala @@ -38,6 +38,7 @@ class CatalystDriver(hconf: HiveConf) extends Driver { false } else { res.addAll(result._2) + result = null true } } diff --git a/src/main/scala/shark/SharkCliDriver.scala b/src/main/scala/shark/SharkCliDriver.scala index 87f55a96..2bf98944 100755 --- a/src/main/scala/shark/SharkCliDriver.scala +++ b/src/main/scala/shark/SharkCliDriver.scala @@ -50,7 +50,7 @@ object SharkCliDriver { private var prompt = "catalyst" private var prompt2 = " " // when ';' is not yet seen. -// private var transport:TSocket = _ + private var transport:TSocket = _ installSignalHandler() @@ -66,10 +66,10 @@ object SharkCliDriver { if (CatalystEnv.sc != null) { CatalystEnv.sc.cancelAllJobs() } else { -// if (transport != null) { -// // Force closing of TCP connection upon session termination -// transport.getSocket().close() -// } + if (transport != null) { + // Force closing of TCP connection upon session termination + transport.getSocket().close() + } } } }) @@ -213,7 +213,7 @@ object SharkCliDriver { val clientTransportTSocketField = classOf[CliSessionState].getDeclaredField("transport") clientTransportTSocketField.setAccessible(true) -// transport = clientTransportTSocketField.get(ss).asInstanceOf[TSocket] + transport = clientTransportTSocketField.get(ss).asInstanceOf[TSocket] var ret = 0 @@ -231,11 +231,11 @@ object SharkCliDriver { line = prefix + line ret = cli.processLine(line, true) prefix = "" - val sharkMode = SharkConfVars.getVar(conf, SharkConfVars.EXEC_MODE) == "shark" + val sharkMode = SharkConfVars.getVar(conf, SharkConfVars.EXEC_MODE) == "catalyst" curPrompt = if (sharkMode) SharkCliDriver.prompt else CliDriver.prompt } else { prefix = prefix + line - val sharkMode = SharkConfVars.getVar(conf, SharkConfVars.EXEC_MODE) == "shark" + val sharkMode = SharkConfVars.getVar(conf, SharkConfVars.EXEC_MODE) == "catalyst" curPrompt = if (sharkMode) SharkCliDriver.prompt2 else CliDriver.prompt2 curPrompt += dbSpaces } @@ -285,7 +285,7 @@ class SharkCliDriver(reloadRdds: Boolean = true) extends CliDriver with LogHelpe tokens(0).equalsIgnoreCase("source") || cmd_trimmed.startsWith("!") || tokens(0).toLowerCase().equals("list") || - ss.asInstanceOf[CliSessionState].isRemoteMode()) { + ss.isRemoteMode()) { val start = System.currentTimeMillis() super.processCmd(cmd) val end = System.currentTimeMillis() @@ -314,6 +314,9 @@ class SharkCliDriver(reloadRdds: Boolean = true) extends CliDriver with LogHelpe proc.asInstanceOf[Driver] } + // TODO HiveContext shouldn't call the SessionState.start(), which has a conflict + // with CliSessionState. Here is a work around to set it(CliSessionState) back. + SessionState.start(ss) logInfo("Execution Mode: " + SharkConfVars.getVar(conf, SharkConfVars.EXEC_MODE)) qp.init() From 0477652ad40f5b292c4dc4293de509b026f3f955 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Wed, 4 Jun 2014 09:27:22 +0800 Subject: [PATCH 04/21] fix bug of cli prompt when switch to hive --- src/main/scala/shark/SharkCliDriver.scala | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/src/main/scala/shark/SharkCliDriver.scala b/src/main/scala/shark/SharkCliDriver.scala index 2bf98944..45e5c7f6 100755 --- a/src/main/scala/shark/SharkCliDriver.scala +++ b/src/main/scala/shark/SharkCliDriver.scala @@ -232,11 +232,19 @@ object SharkCliDriver { ret = cli.processLine(line, true) prefix = "" val sharkMode = SharkConfVars.getVar(conf, SharkConfVars.EXEC_MODE) == "catalyst" - curPrompt = if (sharkMode) SharkCliDriver.prompt else CliDriver.prompt + curPrompt = if (sharkMode) { + SharkCliDriver.prompt + } else { + conf.getVar(HiveConf.ConfVars.CLIPROMPT) + } } else { prefix = prefix + line - val sharkMode = SharkConfVars.getVar(conf, SharkConfVars.EXEC_MODE) == "catalyst" - curPrompt = if (sharkMode) SharkCliDriver.prompt2 else CliDriver.prompt2 + val mode = SharkConfVars.getVar(conf, SharkConfVars.EXEC_MODE) + curPrompt = if (mode == "catalyst") { + SharkCliDriver.prompt2 + } else { + spacesForStringMethod.invoke(null, mode).asInstanceOf[String] + } curPrompt += dbSpaces } line = reader.readLine(curPrompt + "> ") From 0afbc0f24b3d687a0e397013be069549d993c836 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Wed, 4 Jun 2014 09:34:26 +0800 Subject: [PATCH 05/21] update readme --- README.md | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 9c9782ed..6c7a81a1 100755 --- a/README.md +++ b/README.md @@ -59,10 +59,20 @@ resultSet.next() println(resultSet.getInt(1)) ``` +## Running Shark CLI +* Configure the shark_home/conf/shark-env.sh +* Configure the shark_home/conf/hive-site.xml +* Start the Shark CLI +``` +$ bin/shark +catalyst> show tables; +catalyst> set shark.exec.mode=hive; +hive>show tables; +``` + ## Known Missing Features -* Shark CLI * Restoring cached tables upon restart * Invalidation of cached tables when data is INSERTed * Off-heap storage using Tachyon * TGFs -* ... \ No newline at end of file +* ... From ef29e99b71406007af84dfc2ab3946d212fe0cd2 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Wed, 4 Jun 2014 10:35:44 +0800 Subject: [PATCH 06/21] Fix bug of getting schema info --- src/main/scala/shark/CatalystContext.scala | 42 +++++++++++----------- src/main/scala/shark/CatalystDriver.scala | 25 ++++++++----- src/main/scala/shark/SharkCliDriver.scala | 13 +++++-- 3 files changed, 47 insertions(+), 33 deletions(-) diff --git a/src/main/scala/shark/CatalystContext.scala b/src/main/scala/shark/CatalystContext.scala index d636e598..760dee33 100644 --- a/src/main/scala/shark/CatalystContext.scala +++ b/src/main/scala/shark/CatalystContext.scala @@ -42,44 +42,44 @@ import org.apache.spark.sql.execution.QueryExecutionException import shark.LogHelper case class CatalystContext(sc: SparkContext) extends HiveContext(sc) with LogHelper { - private var result: SchemaRDD = _ - class HiveQLQueryExecution(hql: String) extends QueryExecution { override def logical: LogicalPlan = HiveQl.parseSql(hql) override def toString = hql + "\n" + super.toString - def result(): (Int, Seq[String]) = analyzed match { + def result(): (Int, Seq[String], Throwable) = analyzed match { case NativeCommand(cmd) => runOnHive(cmd) - case ExplainCommand(plan) => (0, new QueryExecution { val logical = plan }.toString.split("\n")) + case ExplainCommand(plan) => + (0, new QueryExecution { val logical = plan }.toString.split("\n"), null) case query => try{ val result: Seq[Seq[Any]] = toRdd.collect().toSeq // We need the types so we can output struct field names val types = analyzed.output.map(_.dataType) // Reformat to match hive tab delimited output. - (0, result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t")).toSeq) + (0, result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t")).toSeq, null) } catch { case e: Throwable => { logError("Error:\n $cmd\n", e) - (-1, Seq[String]()) + (-1, Seq[String](), e) } } } - } - - def getResultSetSchema: TableSchema = { - logger.warn(s"Result Schema: ${result.queryExecution.analyzed.output}") - if (result.queryExecution.analyzed.output.size == 0) { - new TableSchema(new FieldSchema("Result", "string", "") :: Nil) - } else { - val schema = result.queryExecution.analyzed.output.map { attr => - new FieldSchema(attr.name, org.apache.spark.sql.hive.HiveMetastoreTypes.toMetastoreType(attr.dataType), "") + + def getResultSetSchema: TableSchema = { + logger.warn(s"Result Schema: ${analyzed.output}") + if (analyzed.output.size == 0) { + new TableSchema(new FieldSchema("Result", "string", "") :: Nil) + } else { + val schema = analyzed.output.map { attr => + new FieldSchema(attr.name, + org.apache.spark.sql.hive.HiveMetastoreTypes.toMetastoreType(attr.dataType), "") + } + new TableSchema(schema) } - new TableSchema(schema) } } - def runOnHive(cmd: String, maxRows: Int = 1000): (Int, Seq[String]) = { + def runOnHive(cmd: String, maxRows: Int = 1000): (Int, Seq[String], Throwable) = { try { val cmd_trimmed: String = cmd.trim() val tokens: Array[String] = cmd_trimmed.split("\\s+") @@ -100,13 +100,13 @@ case class CatalystContext(sc: SparkContext) extends HiveContext(sc) with LogHel driver.setMaxRows(maxRows) driver.getResults(results) driver.destroy() - (0, results) + (0, results, null) case _ => SessionState.get().out.println(tokens(0) + " " + cmd_1) - (proc.run(cmd_1).getResponseCode, Seq[String]()) + (proc.run(cmd_1).getResponseCode, Seq[String](), null) } } catch { - case e: Exception => + case e: Throwable => logger.error( s""" |====================== @@ -117,7 +117,7 @@ case class CatalystContext(sc: SparkContext) extends HiveContext(sc) with LogHel |END HIVE FAILURE OUTPUT |====================== """.stripMargin) - throw e + (-2, Seq[String](), null) } } } diff --git a/src/main/scala/shark/CatalystDriver.scala b/src/main/scala/shark/CatalystDriver.scala index bb731474..b21428c3 100644 --- a/src/main/scala/shark/CatalystDriver.scala +++ b/src/main/scala/shark/CatalystDriver.scala @@ -3,35 +3,42 @@ package shark import java.util.ArrayList import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.metastore.api.Schema +import org.apache.hive.service.cli.TableSchema import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse import org.apache.hadoop.hive.conf.HiveConf - import org.apache.spark.sql.hive.CatalystContext - import scala.collection.JavaConversions._ +import org.apache.commons.lang.exception.ExceptionUtils class CatalystDriver(hconf: HiveConf) extends Driver { private val context: CatalystContext = CatalystEnv.cc - private var schema: Schema = _ - private var result: (Int, Seq[String]) = _ + private var tschema: TableSchema = _ + private var result: (Int, Seq[String], Throwable) = _ override def init(): Unit = { } override def run(command: String): CommandProcessorResponse = { - this.result = new context.HiveQLQueryExecution(command).result + val execution = new context.HiveQLQueryExecution(command) + result = execution.result + tschema = execution.getResultSetSchema - new CommandProcessorResponse(this.result._1) + if(result._1 != 0) { + new CommandProcessorResponse(result._1, ExceptionUtils.getStackTrace(result._3), null) + } else { + new CommandProcessorResponse(result._1) + } } override def close(): Int = { result = null - schema = null + tschema = null 0 } - override def getSchema(): Schema = schema + override def getSchema(): Schema = throw new UnsupportedOperationException("for getSchema") + def getTableSchema = tschema override def getResults(res: ArrayList[String]): Boolean = { if(result == null) { @@ -45,6 +52,6 @@ class CatalystDriver(hconf: HiveConf) extends Driver { override def destroy() { result = null - schema = null + tschema = null } } \ No newline at end of file diff --git a/src/main/scala/shark/SharkCliDriver.scala b/src/main/scala/shark/SharkCliDriver.scala index 45e5c7f6..e118cfdf 100755 --- a/src/main/scala/shark/SharkCliDriver.scala +++ b/src/main/scala/shark/SharkCliDriver.scala @@ -344,9 +344,16 @@ class SharkCliDriver(reloadRdds: Boolean = true) extends CliDriver with LogHelpe if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_CLI_PRINT_HEADER)) { // Print the column names. - val fieldSchemas = qp.getSchema.getFieldSchemas - if (fieldSchemas != null) { - out.println(fieldSchemas.map(_.getName).mkString("\t")) + if(qp.isInstanceOf[CatalystDriver]) { + val fieldDescs = qp.asInstanceOf[CatalystDriver].getTableSchema.getColumnDescriptors() + if (fieldDescs != null) { + out.println(fieldDescs.map(_.getName()).mkString("\t")) + } + } else { + val fieldSchemas = qp.getSchema.getFieldSchemas + if (fieldSchemas != null) { + out.println(fieldSchemas.map(_.getName).mkString("\t")) + } } } From 6c1d9f5c9ae9a4ac28250af78b83d5175d4931e9 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Wed, 4 Jun 2014 11:01:51 +0800 Subject: [PATCH 07/21] Add bug info in the README --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index 6c7a81a1..35bb025c 100755 --- a/README.md +++ b/README.md @@ -69,6 +69,7 @@ catalyst> show tables; catalyst> set shark.exec.mode=hive; hive>show tables; ``` +But there is a bug, which require show tables before doing anything else. ## Known Missing Features * Restoring cached tables upon restart From 3d344d0d5c2a7004d960478d80cf3f8b23b03427 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Wed, 4 Jun 2014 11:09:34 +0800 Subject: [PATCH 08/21] remove the mistaken commit --- src/main/test/0.20S-exclude.txt | 9 - src/main/test/0.20S-include.txt | 48 - src/main/test/README.md | 28 - src/main/test/java/shark/JavaAPISuite.java | 120 -- src/main/test/java/shark/SharkQTestUtil.java | 188 --- .../test/java/shark/TestSharkCliDriver.java | 146 --- src/main/test/scala/shark/CliSuite.scala | 62 - .../scala/shark/ColumnStatsSQLSuite.scala | 129 -- .../shark/CountDistinctRewriteSuite.scala | 283 ---- .../test/scala/shark/ReflectionSuite.scala | 112 -- src/main/test/scala/shark/SQLSuite.scala | 1154 ----------------- src/main/test/scala/shark/SharkRunner.scala | 127 -- .../test/scala/shark/SharkServerSuite.scala | 108 -- src/main/test/scala/shark/SortSuite.scala | 53 - .../test/scala/shark/TachyonSQLSuite.scala | 436 ------- src/main/test/scala/shark/TestUtils.scala | 195 --- src/main/test/scala/shark/UtilsSuite.scala | 55 - .../execution/HiveStructSerializerSuite.scala | 65 - .../serialization/SerializationSuite.scala | 109 -- .../ShuffleSerializerSuite.scala | 112 -- .../shark/memstore2/CachePolicySuite.scala | 131 -- .../shark/memstore2/ColumnIteratorSuite.scala | 413 ------ .../shark/memstore2/ColumnStatsSuite.scala | 232 ---- .../shark/memstore2/TablePartitionSuite.scala | 121 -- .../memstore2/column/ColumnTypeSuite.scala | 121 -- .../CompressedColumnIteratorSuite.scala | 243 ---- .../column/CompressionAlgorithmSuite.scala | 366 ------ .../column/NullableColumnBuilderSuite.scala | 145 --- .../column/NullableColumnIteratorSuite.scala | 118 -- .../scala/shark/util/BloomFilterSuite.scala | 35 - src/main/test/tests_fail.txt | 232 ---- src/main/test/tests_pass.txt | 925 ------------- 32 files changed, 6621 deletions(-) delete mode 100755 src/main/test/0.20S-exclude.txt delete mode 100755 src/main/test/0.20S-include.txt delete mode 100755 src/main/test/README.md delete mode 100755 src/main/test/java/shark/JavaAPISuite.java delete mode 100755 src/main/test/java/shark/SharkQTestUtil.java delete mode 100755 src/main/test/java/shark/TestSharkCliDriver.java delete mode 100755 src/main/test/scala/shark/CliSuite.scala delete mode 100755 src/main/test/scala/shark/ColumnStatsSQLSuite.scala delete mode 100755 src/main/test/scala/shark/CountDistinctRewriteSuite.scala delete mode 100755 src/main/test/scala/shark/ReflectionSuite.scala delete mode 100755 src/main/test/scala/shark/SQLSuite.scala delete mode 100755 src/main/test/scala/shark/SharkRunner.scala delete mode 100755 src/main/test/scala/shark/SharkServerSuite.scala delete mode 100755 src/main/test/scala/shark/SortSuite.scala delete mode 100755 src/main/test/scala/shark/TachyonSQLSuite.scala delete mode 100755 src/main/test/scala/shark/TestUtils.scala delete mode 100755 src/main/test/scala/shark/UtilsSuite.scala delete mode 100755 src/main/test/scala/shark/execution/HiveStructSerializerSuite.scala delete mode 100755 src/main/test/scala/shark/execution/serialization/SerializationSuite.scala delete mode 100755 src/main/test/scala/shark/execution/serialization/ShuffleSerializerSuite.scala delete mode 100755 src/main/test/scala/shark/memstore2/CachePolicySuite.scala delete mode 100755 src/main/test/scala/shark/memstore2/ColumnIteratorSuite.scala delete mode 100755 src/main/test/scala/shark/memstore2/ColumnStatsSuite.scala delete mode 100755 src/main/test/scala/shark/memstore2/TablePartitionSuite.scala delete mode 100755 src/main/test/scala/shark/memstore2/column/ColumnTypeSuite.scala delete mode 100755 src/main/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala delete mode 100755 src/main/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala delete mode 100755 src/main/test/scala/shark/memstore2/column/NullableColumnBuilderSuite.scala delete mode 100755 src/main/test/scala/shark/memstore2/column/NullableColumnIteratorSuite.scala delete mode 100755 src/main/test/scala/shark/util/BloomFilterSuite.scala delete mode 100755 src/main/test/tests_fail.txt delete mode 100755 src/main/test/tests_pass.txt diff --git a/src/main/test/0.20S-exclude.txt b/src/main/test/0.20S-exclude.txt deleted file mode 100755 index 92a75082..00000000 --- a/src/main/test/0.20S-exclude.txt +++ /dev/null @@ -1,9 +0,0 @@ -testCliDriver_archive_excludeHadoop20 -testCliDriver_auto_join14 -testCliDriver_combine2 -testCliDriver_ctas -testCliDriver_input12 -testCliDriver_input39 -testCliDriver_join14 -testCliDriver_loadpart_err -testCliDriver_sample_islocalmode_hook \ No newline at end of file diff --git a/src/main/test/0.20S-include.txt b/src/main/test/0.20S-include.txt deleted file mode 100755 index d15b64ef..00000000 --- a/src/main/test/0.20S-include.txt +++ /dev/null @@ -1,48 +0,0 @@ -testCliDriver_archive -testCliDriver_archive_corrupt -testCliDriver_infer_bucket_sort_list_bucket -testCliDriver_list_bucket_dml_1 -testCliDriver_list_bucket_dml_11 -testCliDriver_list_bucket_dml_12 -testCliDriver_list_bucket_dml_13 -testCliDriver_list_bucket_dml_2 -testCliDriver_list_bucket_dml_3 -testCliDriver_list_bucket_dml_4 -testCliDriver_list_bucket_dml_5 -testCliDriver_list_bucket_dml_6 -testCliDriver_list_bucket_dml_7 -testCliDriver_list_bucket_dml_8 -testCliDriver_list_bucket_dml_9 -testCliDriver_list_bucket_query_multiskew_1 -testCliDriver_list_bucket_query_multiskew_2 -testCliDriver_list_bucket_query_multiskew_3 -testCliDriver_list_bucket_query_oneskew_1 -testCliDriver_list_bucket_query_oneskew_2 -testCliDriver_list_bucket_query_oneskew_3 -testCliDriver_recursive_dir -testCliDriver_skewjoin_union_remove_1 -testCliDriver_skewjoin_union_remove_2 -testCliDriver_split_sample -testCliDriver_union_remove_1 -testCliDriver_union_remove_10 -testCliDriver_union_remove_11 -testCliDriver_union_remove_12 -testCliDriver_union_remove_13 -testCliDriver_union_remove_14 -testCliDriver_union_remove_15 -testCliDriver_union_remove_16 -testCliDriver_union_remove_17 -testCliDriver_union_remove_18 -testCliDriver_union_remove_19 -testCliDriver_union_remove_2 -testCliDriver_union_remove_20 -testCliDriver_union_remove_21 -testCliDriver_union_remove_22 -testCliDriver_union_remove_23 -testCliDriver_union_remove_24 -testCliDriver_union_remove_3 -testCliDriver_union_remove_4 -testCliDriver_union_remove_5 -testCliDriver_union_remove_7 -testCliDriver_union_remove_8 -testCliDriver_union_remove_9 \ No newline at end of file diff --git a/src/main/test/README.md b/src/main/test/README.md deleted file mode 100755 index 1ae3b814..00000000 --- a/src/main/test/README.md +++ /dev/null @@ -1,28 +0,0 @@ -###Hive Compatibility Test Warnings - -#### Test results that rely on tables with `timestamp` fields may differ across JVM versions. -For example, these tests: -* udf5 -* timestamp.1, timestamp_2, timestamp_udf - -Pass when running with this JVM: -(Mac 10.9, AMPLab Jenkins) -java version "1.7.0_25" -Java(TM) SE Runtime Environment (build 1.7.0_25-b15) -Java HotSpot(TM) 64-Bit Server VM (build 23.25-b01, mixed mode) - -But fail on EC2 when run with this JVM: -(EC2 c2.2xlarge) -java version "1.7.0_45" -OpenJDK Runtime Environment (amzn-2.4.3.2.32.amzn1-x86_64 u45-b15) -OpenJDK 64-Bit Server VM (build 24.45-b08, mixed mode) - - -A few more tests from test_pass.txt that fall into this category: -TestCliDriver_input_part8 -TestSharkCliDriver: testCliDriver_timestamp_1 -TestSharkCliDriver: testCliDriver_timestamp_2 -TestSharkCliDriver: testCliDriver_timestamp_3 -TestSharkCliDriver: testCliDriver_timestamp_udf -TestSharkCliDriver: testCliDriver_udf_to_unix_timestamp -TestSharkCliDriver: testCliDriver_udf5 diff --git a/src/main/test/java/shark/JavaAPISuite.java b/src/main/test/java/shark/JavaAPISuite.java deleted file mode 100755 index 8e86890d..00000000 --- a/src/main/test/java/shark/JavaAPISuite.java +++ /dev/null @@ -1,120 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark; - -import java.io.Serializable; -import java.util.List; - -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; - -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.function.Function; - -import shark.api.Row; -import shark.api.JavaSharkContext; -import shark.api.JavaTableRDD; - - -// The test suite itself is Serializable so that anonymous Function implementations can be -// serialized, as an alternative to converting these anonymous classes to static inner classes; -// see http://stackoverflow.com/questions/758570/. -public class JavaAPISuite implements Serializable { - - private static final String WAREHOUSE_PATH = TestUtils$.MODULE$.getWarehousePath(); - private static final String METASTORE_PATH = TestUtils$.MODULE$.getMetastorePath(); - - private static transient JavaSharkContext sc; - - @BeforeClass - public static void oneTimeSetUp() { - // Intentionally leaving this here since SBT doesn't seem to display junit tests well ... - System.out.println("running JavaAPISuite ================================================"); - - // Check if the SharkEnv's SharkContext has already been initialized. If so, use that to - // instantiate a JavaSharkContext. - sc = SharkRunner.initWithJava(); - - // test - sc.sql("drop table if exists test_java"); - sc.sql("CREATE TABLE test_java (key INT, val STRING)"); - sc.sql("LOAD DATA LOCAL INPATH '${hiveconf:shark.test.data.path}/kv1.txt' INTO TABLE test_java"); - - // users - sc.sql("drop table if exists users_java"); - sc.sql("create table users_java (id int, name string) row format delimited fields terminated by '\t'"); - sc.sql("load data local inpath '${hiveconf:shark.test.data.path}/users.txt' OVERWRITE INTO TABLE users_java"); - } - - @AfterClass - public static void oneTimeTearDown() { - sc.stop(); - System.clearProperty("spark.driver.port"); - } - - @Test - public void selectQuery() { - List result = sc.sql("select val from test_java"); - Assert.assertEquals(500, result.size()); - Assert.assertTrue(result.contains("val_407")); - } - - @Test - public void sql2rdd() { - JavaTableRDD result = sc.sql2rdd("select val from test_java"); - JavaRDD values = result.map(new Function() { - @Override - public String call(Row x) { - return x.getString(0); - } - }); - Assert.assertEquals(500, values.count()); - Assert.assertTrue(values.collect().contains("val_407")); - } - - @Test - public void filter() { - JavaTableRDD result = sc.sql2rdd("select * from users_java"); - JavaTableRDD filtered = result.filter(new Function() { - @Override - public Boolean call(Row row) throws Exception { - return row.getString("name").equals("B"); - } - }).cache(); - Assert.assertEquals(1, filtered.count()); - Assert.assertEquals(2, filtered.first().getInt("id").intValue()); - } - - @Test - public void union() { - JavaTableRDD a = sc.sql2rdd("select * from users_java where name = \"A\""); - JavaTableRDD b = sc.sql2rdd("select * from users_java where name = \"B\""); - JavaTableRDD union = a.union(b); - Assert.assertEquals(3, union.count()); - List uniqueNames = union.map(new Function() { - @Override - public String call(Row row) throws Exception { - return row.getString("name"); - } - }).distinct().collect(); - Assert.assertEquals(2, uniqueNames.size()); - } - -} diff --git a/src/main/test/java/shark/SharkQTestUtil.java b/src/main/test/java/shark/SharkQTestUtil.java deleted file mode 100755 index 67f69fe1..00000000 --- a/src/main/test/java/shark/SharkQTestUtil.java +++ /dev/null @@ -1,188 +0,0 @@ -package shark; - -import java.io.File; -import java.io.FileOutputStream; -import java.io.PrintStream; -import java.lang.reflect.Method; - -import org.apache.commons.lang.StringUtils; -import org.apache.hadoop.hive.cli.CliSessionState; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.QTestUtil; -import org.apache.hadoop.hive.ql.exec.Utilities.StreamPrinter; -import org.apache.hadoop.hive.ql.session.SessionState; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -/** - * Replaces Hive's QTestUtil class by using the SharkDriver instead of Hive's - * Driver. Also changes the way comparison is done by forcing a sort and - * truncating floating point numbers. - */ -public class SharkQTestUtil extends QTestUtil { - - private static Method maskPatternsMethod; - - static final private Log LOG = LogFactory.getLog(SharkQTestUtil.class.getName()); - - static { - try { - maskPatternsMethod = QTestUtil.class.getDeclaredMethod("maskPatterns", - String[].class, String.class); - maskPatternsMethod.setAccessible(true); - } catch (SecurityException e) { - e.printStackTrace(); - } catch (NoSuchMethodException e) { - e.printStackTrace(); - } - } - - private SharkCliDriver cliDrv; - - public SharkQTestUtil(String outDir, String logDir) throws Exception { - super(outDir, logDir); - } - - public SharkQTestUtil(String outDir, String logDir, boolean miniMr, - String hadoopVer) throws Exception { - super(outDir, logDir, miniMr, hadoopVer); - } - - @Override - public void cliInit(String tname, boolean recreate) throws Exception { - SharkConfVars.setVar(conf, SharkConfVars.EXPLAIN_MODE(), "hive"); - - if (recreate) { - cleanUp(); - createSources(); - } - - HiveConf.setVar(conf, HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER, - "org.apache.hadoop.hive.ql.security.DummyAuthenticator"); - - // Set the default min number of map tasks to 1 since that's what most Hive tests expects. - conf.setInt("mapred.map.tasks", 1); - - CliSessionState ss = new CliSessionState(conf); - assert(ss != null); - ss.in = System.in; - - File qf = new File(outDir, tname); - File outf = new File(new File(logDir), qf.getName().concat(".out")); - FileOutputStream fo = new FileOutputStream(outf); - ss.out = new PrintStream(fo, true, "UTF-8"); - ss.err = ss.out; - ss.setIsSilent(true); - SessionState oldSs = SessionState.get(); - if (oldSs != null && oldSs.out != null && oldSs.out != System.out) { - oldSs.out.close(); - } - SessionState.start(ss); - - cliDrv = new SharkCliDriver(); - if (tname.equals("init_file.q")) { - File testInitFile = new File("../data/scripts/test_init_file.sql"); - try { - ss.initFiles.add(testInitFile.getAbsolutePath()); - } catch (Exception e) { - System.out.println("Exception is =" + e.getMessage()); - } - } - cliDrv.processInitFiles(ss); - } - - @Override - public int executeClient(String tname) { - LOG.info("Begin query: " + tname); - return cliDrv.processLine(getQMap().get(tname)); - } - - @Override - public int checkCliDriverResults(String tname) throws Exception { - String[] cmdArray; - String[] patterns; - assert(getQMap().containsKey(tname)); - - String outFileName = outPath(outDir, tname + ".out"); - - patterns = new String[] { - ".*file:.*", - ".*pfile:.*", - ".*hdfs:.*", - ".*/tmp/.*", - ".*invalidscheme:.*", - ".*lastUpdateTime.*", - ".*lastAccessTime.*", - ".*lastModifiedTime.*", - ".*[Oo]wner.*", - ".*CreateTime.*", - ".*LastAccessTime.*", - ".*Location.*", - ".*LOCATION '.*", - ".*transient_lastDdlTime.*", - ".*last_modified_.*", - ".*java.lang.RuntimeException.*", - ".*at org.*", - ".*at sun.*", - ".*at java.*", - ".*at junit.*", - ".*Caused by:.*", - ".*LOCK_QUERYID:.*", - ".*LOCK_TIME:.*", - ".*grantTime.*", - ".*[.][.][.] [0-9]* more.*", - ".*job_[0-9]*_[0-9]*.*", - ".*USING 'java -cp.*", - "^Deleted.*", - }; - maskPatternsMethod.invoke(this, patterns, (new File(logDir, tname + ".out")).getPath()); - - cmdArray = new String[] { - "diff", "-a", - "-I", "PREHOOK", - "-I", "POSTHOOK" - }; - - // Only keep 5 digits of precision for floating point numbers. - // Also trim trailing whitespace. - String truncFloatCmd = "perl -p -e 's/(\\d\\.\\d{5})\\d*/\\1/g;' -e 's/\\s+$/\\n/g'"; - String expectedFile = (new File(logDir, tname + ".out")).getPath(); - - String cmdString = "\"" - + StringUtils.join(cmdArray, "\" \"") + "\" " - + "<(sort " + expectedFile + " | " + truncFloatCmd + ") " - + "<(sort " + outFileName + " | " + truncFloatCmd + ")"; - System.out.println("Comparing: " + expectedFile + " " + outFileName); - System.out.println(cmdString); - - //System.out.println(org.apache.commons.lang.StringUtils.join(cmdArray, ' ')); - String[] bashCmdArray = new String[3]; - bashCmdArray[0] = "bash"; - bashCmdArray[1] = "-c"; - bashCmdArray[2] = cmdString; - Process executor = Runtime.getRuntime().exec(bashCmdArray); - - StreamPrinter outPrinter = new StreamPrinter( - executor.getInputStream(), null, SessionState.getConsole().getChildOutStream()); - StreamPrinter errPrinter = new StreamPrinter( - executor.getErrorStream(), null, SessionState.getConsole().getChildErrStream()); - - outPrinter.start(); - errPrinter.start(); - - int exitVal = executor.waitFor(); - - if (exitVal != 0 && overWrite) { - System.out.println("Overwriting results"); - cmdArray = new String[3]; - cmdArray[0] = "cp"; - cmdArray[1] = (new File(logDir, tname + ".out")).getPath(); - cmdArray[2] = outFileName; - executor = Runtime.getRuntime().exec(cmdArray); - exitVal = executor.waitFor(); - } - - return exitVal; - } - -} diff --git a/src/main/test/java/shark/TestSharkCliDriver.java b/src/main/test/java/shark/TestSharkCliDriver.java deleted file mode 100755 index a455092f..00000000 --- a/src/main/test/java/shark/TestSharkCliDriver.java +++ /dev/null @@ -1,146 +0,0 @@ -package shark; - -import java.io.BufferedReader; -import java.io.FileNotFoundException; -import java.io.FileReader; -import java.io.IOException; -import java.lang.reflect.Field; -import java.util.Enumeration; -import java.util.HashSet; -import java.util.Set; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -import junit.framework.Test; -import junit.framework.TestCase; -import junit.framework.TestSuite; -import org.junit.Ignore; - -import org.apache.hadoop.hive.cli.TestCliDriver; -import org.apache.hadoop.hive.ql.QTestUtil; - -import java.util.Iterator; - -/** - * The test driver. It overloads Hive's TestCliDriver to use SharkQTestUtil. - * There is also a feature to selectively run tests, i.e. only tests whose - * names match the regular expression pattern defined in environmental variable - * TEST are invoked. Note that this test is ignored so we don't run it in our - * normal unit tests. This should only be launched with the explicit test script. - */ -@Ignore public class TestSharkCliDriver extends TestCliDriver { - - static { - // Replace qt in Hive's TestCliDriver with SharkQTestUtil. - try { - - Field qtField = TestCliDriver.class.getDeclaredField("qt"); - qtField.setAccessible(true); - - Field outDirField = QTestUtil.class.getDeclaredField("outDir"); - outDirField.setAccessible(true); - Field logDirField = QTestUtil.class.getDeclaredField("logDir"); - logDirField.setAccessible(true); - - QTestUtil qt = (QTestUtil) qtField.get(null); - String outDir = (String) outDirField.get(qt); - String logDir = (String) logDirField.get(qt); - - qt = new SharkQTestUtil(outDir, logDir); - // cleanUp / createSources are already called in TestCliDriver. - //qt.cleanUp(); - //qt.createSources(); - - qtField.set(null, qt); - - } catch (Exception e) { - e.printStackTrace(); - } - } - - public TestSharkCliDriver(String name) { - super(name); - } - - public static Test suite() { - TestSuite suite = new TestSuite(); - TestSuite hiveSuite = (TestSuite) TestCliDriver.suite(); - - @SuppressWarnings("unchecked") - Enumeration tests = (Enumeration) hiveSuite.tests(); - - String fileName = System.getenv("TEST_FILE"); - Set regTestsFromFile = new HashSet(); - if (fileName != null && fileName.length() > 0) { - try { - BufferedReader reader = new BufferedReader(new FileReader(fileName)); - String line = null; - while ((line = reader.readLine()) != null) { - regTestsFromFile.add(line); - } - } catch (FileNotFoundException e) { - System.err.println("FileNotFoundException: " + e.getMessage()); - System.exit(1); - } catch (IOException e) { - System.err.println("IOException: " + e.getMessage()); - System.exit(1); - } - } - - Pattern regexPattern = null; - String pattern = System.getenv("TEST"); - if (pattern != null && pattern.length() > 0) { - regexPattern = Pattern.compile(System.getenv("TEST")); - } - - System.out.println("---------------------------------------------------"); - System.out.println("---------------------------------------------------"); - System.out.println("---------------------------------------------------"); - System.out.println("---------------------------------------------------"); - System.out.println(TestSharkCliDriver.class.getName()); - - boolean readFile = (regTestsFromFile.size() != 0); - - while (tests.hasMoreElements()) { - TestCase test = (TestCase) tests.nextElement(); - - boolean passRegex = (regexPattern == null); - boolean passFile = !readFile; - - if (regexPattern != null) { - Matcher m = regexPattern.matcher(test.getName()); - if (m.find() || test.getName() == "testCliDriver_shutdown") { - passRegex = true; - } - } - - if (regTestsFromFile.size() > 0) { - passFile = regTestsFromFile.contains(test.getName()); - } - - if (passRegex && passFile) { - suite.addTest(test); - regTestsFromFile.remove(test.getName()); - System.out.println("TestSharkCliDriver: " + test.getName()); - if (readFile && regTestsFromFile.size() == 0) - break; - } - } - - Iterator regTestsFromFileIter = regTestsFromFile.iterator(); - while (regTestsFromFileIter.hasNext()) { - String test = regTestsFromFileIter.next(); - System.out.println("Warning! Hive test not found: " + test); - } - - System.out.println("TestSharkCliDriver total test to run: " + suite.countTestCases()); - - System.out.println("---------------------------------------------------"); - System.out.println("---------------------------------------------------"); - System.out.println("---------------------------------------------------"); - System.out.println("---------------------------------------------------"); - - - return suite; - } -} diff --git a/src/main/test/scala/shark/CliSuite.scala b/src/main/test/scala/shark/CliSuite.scala deleted file mode 100755 index f2e87796..00000000 --- a/src/main/test/scala/shark/CliSuite.scala +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark - -import java.io.{BufferedReader, File, InputStreamReader, PrintWriter} -import org.scalatest.{BeforeAndAfterAll, FunSuite} - - -/** - * Test the Shark CLI. - */ -class CliSuite extends FunSuite with BeforeAndAfterAll with TestUtils { - - val WAREHOUSE_PATH = TestUtils.getWarehousePath("cli") - val METASTORE_PATH = TestUtils.getMetastorePath("cli") - - override def beforeAll() { - val pb = new ProcessBuilder( - "./bin/shark", - "-hiveconf", - "javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + METASTORE_PATH + ";create=true", - "-hiveconf", - "hive.metastore.warehouse.dir=" + WAREHOUSE_PATH) - - process = pb.start() - outputWriter = new PrintWriter(process.getOutputStream, true) - inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) - errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) - waitForOutput(inputReader, "shark>") - } - - override def afterAll() { - process.destroy() - process.waitFor() - } - - test("simple select") { - val dataFilePath = TestUtils.dataFilePath + "/kv1.txt" - executeQuery("create table shark_test1(key int, val string);") - executeQuery("load data local inpath '" + dataFilePath+ "' overwrite into table shark_test1;") - executeQuery("""create table shark_test1_cached TBLPROPERTIES ("shark.cache" = "true") as - select * from shark_test1;""") - val out = executeQuery("select * from shark_test1_cached where key = 407;") - assert(out.contains("val_407")) - } - -} diff --git a/src/main/test/scala/shark/ColumnStatsSQLSuite.scala b/src/main/test/scala/shark/ColumnStatsSQLSuite.scala deleted file mode 100755 index 705496f2..00000000 --- a/src/main/test/scala/shark/ColumnStatsSQLSuite.scala +++ /dev/null @@ -1,129 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark - -import org.apache.hadoop.io.BytesWritable - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite - -import org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME - -import org.apache.spark.rdd.RDD - -import shark.memstore2.MemoryMetadataManager - - -class ColumnStatsSQLSuite extends FunSuite with BeforeAndAfterAll { - - val sc: SharkContext = SharkRunner.init() - val sharkMetastore = SharkEnv.memoryMetadataManager - - // import expectSql() shortcut methods - import shark.SharkRunner._ - - override def beforeAll() { - sc.runSql("drop table if exists srcpart_cached") - sc.runSql("create table srcpart_cached(key int, val string) partitioned by (keypart int)") - sc.runSql("""load data local inpath '${hiveconf:shark.test.data.path}/kv1.txt' - into table srcpart_cached partition (keypart = 1)""") - } - - override def afterAll() { - sc.runSql("drop table if exists srcpart_cached") - } - - test("Hive partition stats are tracked") { - val tableOpt = sharkMetastore.getPartitionedTable(DEFAULT_DATABASE_NAME, "srcpart_cached") - assert(tableOpt.isDefined) - val partitionToStatsOpt = tableOpt.get.getStats("keypart=1") - assert(partitionToStatsOpt.isDefined) - val partitionToStats = partitionToStatsOpt.get - // The 'kv1.txt' file loaded into 'keypart=1' in beforeAll() has 2 partitions. - assert(partitionToStats.size == 2) - } - - test("Hive partition stats are tracked after LOADs and INSERTs") { - // Load more data into srcpart_cached - sc.runSql("""load data local inpath '${hiveconf:shark.test.data.path}/kv1.txt' - into table srcpart_cached partition (keypart = 1)""") - val tableOpt = sharkMetastore.getPartitionedTable(DEFAULT_DATABASE_NAME, "srcpart_cached") - assert(tableOpt.isDefined) - var partitionToStatsOpt = tableOpt.get.getStats("keypart=1") - assert(partitionToStatsOpt.isDefined) - var partitionToStats = partitionToStatsOpt.get - // The 'kv1.txt' file loaded into 'keypart=1' has 2 partitions. We've loaded it twice at this - // point. - assert(partitionToStats.size == 4) - - // Append using INSERT command - sc.runSql("insert into table srcpart_cached partition(keypart = 1) select * from test") - partitionToStatsOpt = tableOpt.get.getStats("keypart=1") - assert(partitionToStatsOpt.isDefined) - partitionToStats = partitionToStatsOpt.get - assert(partitionToStats.size == 6) - - // INSERT OVERWRITE should overrwritie old table stats. This also restores srcpart_cached - // to contents contained before this test. - sc.runSql("""insert overwrite table srcpart_cached partition(keypart = 1) - select * from test""") - partitionToStatsOpt = tableOpt.get.getStats("keypart=1") - assert(partitionToStatsOpt.isDefined) - partitionToStats = partitionToStatsOpt.get - assert(partitionToStats.size == 2) - } - - ////////////////////////////////////////////////////////////////////////////// - // End-to-end sanity checks - ////////////////////////////////////////////////////////////////////////////// - test("column pruning filters") { - expectSql("select count(*) from test_cached where key > -1", "500") - } - - test("column pruning group by") { - expectSql("select key, count(*) from test_cached group by key order by key limit 1", "0\t3") - } - - test("column pruning group by with single filter") { - expectSql("select key, count(*) from test_cached where val='val_484' group by key", "484\t1") - } - - test("column pruning aggregate function") { - expectSql("select val, sum(key) from test_cached group by val order by val desc limit 1", - "val_98\t196") - } - - test("column pruning filters for a Hive partition") { - expectSql("select count(*) from srcpart_cached where key > -1", "500") - expectSql("select count(*) from srcpart_cached where key > -1 and keypart = 1", "500") - } - - test("column pruning group by for a Hive partition") { - expectSql("select key, count(*) from srcpart_cached group by key order by key limit 1", "0\t3") - } - - test("column pruning group by with single filter for a Hive partition") { - expectSql("select key, count(*) from srcpart_cached where val='val_484' group by key", "484\t1") - } - - test("column pruning aggregate function for a Hive partition") { - expectSql("select val, sum(key) from srcpart_cached group by val order by val desc limit 1", - "val_98\t196") - } - -} diff --git a/src/main/test/scala/shark/CountDistinctRewriteSuite.scala b/src/main/test/scala/shark/CountDistinctRewriteSuite.scala deleted file mode 100755 index 04ef2f55..00000000 --- a/src/main/test/scala/shark/CountDistinctRewriteSuite.scala +++ /dev/null @@ -1,283 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark - -import java.util.{List => JavaList} - -import scala.collection.JavaConversions._ - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite - -import org.apache.hadoop.hive.ql.lib.Node -import org.apache.hadoop.hive.ql.parse.{ASTNode, HiveParser} -import org.apache.hadoop.hive.ql.parse.{ParseDriver, ParseUtils} - -import shark.api.QueryExecutionException -import shark.parse.ASTRewriteUtil -import shark.parse.ASTRewriteUtil._ - - -class CountDistinctRewriteSuite extends FunSuite with BeforeAndAfterAll { - - /** - * Throws an error if this is not equal to other. - * - * Right now this function only checks the name, type, text and children of the node - * for equality. - */ - def checkEquals(node: ASTNode, other: ASTNode) { - def check(field: String, func: ASTNode => Any) { - assert(func(node) == func(other), - "node: \n" + printTree(node) + "\nis not equal to: \n" + printTree(other) + - "\n for field: " + field) - } - - check("name", _.getName) - check("type", _.getType) - check("text", _.getText) - check("numChildren", (node: ASTNode) => getChildren(node).size) - - val leftChildren = getChildren(node) - val rightChildren = getChildren(other) - leftChildren.zip(rightChildren).foreach { - case (l,r) => checkEquals(l, r) - } - } - - def genAST(command: String): ASTNode = { - try { - ParseUtils.findRootNonNullToken((new ParseDriver()).parse(command)) - } catch { - case e: Exception => - throw new RuntimeException("Failed to parse: " + command) - } - } - - test("Count distinct, single column") { - val command = genAST("select count(distinct key) from src") - val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) - val expectedRewrite = genAST("select count(*) from (select distinct key from src) %s" - .format(ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "0")) - checkEquals(rewrite, expectedRewrite) - } - - test("Count distinct, multiple columns") { - val command = genAST("select count(distinct key, value) from src") - val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) - val expectedRewrite = genAST("select count(*) from (select distinct key, value from src) %s" - .format(ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "0")) - checkEquals(rewrite, expectedRewrite) - } - - test("Multiple columns with expressions") { - val command = genAST("select count(distinct key * 10 - 3, substr(value, 5)) from src") - val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) - val expectedRewrite = genAST( - "select count(*) from (select distinct key * 10 - 3, substr(value, 5) from src) %s" - .format(ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "0")) - checkEquals(rewrite, expectedRewrite) - } - - test("Distinct function outputs") { - val command = genAST("select count(distinct substr(val, 5)) from src") - val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) - val expectedRewrite = genAST(""" - select count(*) from (select distinct substr(val, 5) from src) %s""" - .format(ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "0")) - checkEquals(rewrite, expectedRewrite) - } - - test("Constants aside COUNT DISTINCT in SELECT list") { - val command1 = genAST("select 1, 2, count(distinct key) from src") - val rewrite1 = ASTRewriteUtil.countDistinctToGroupBy(command1) - val expectedRewrite1 = genAST(""" - select 1, 2, count(*) from (select distinct key from src) %s""" - .format(ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "0")) - checkEquals(rewrite1, expectedRewrite1) - - val command2 = genAST("select 1, count(distinct key), 2, 3 from src") - val rewrite2 = ASTRewriteUtil.countDistinctToGroupBy(command2) - val expectedRewrite2 = genAST(""" - select 1, count(*), 2, 3 from (select distinct key from src) %s""" - .format(ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "0")) - checkEquals(rewrite2, expectedRewrite2) - } - - test("COUNT DISTINCT as part of an expression") { - val command = genAST("select count(distinct key) + 10 from src") - val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) - val expectedRewrite = genAST(""" - select count(*) + 10 from (select distinct key from src) %s""" - .format(ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "0")) - checkEquals(rewrite, expectedRewrite) - } - - test("COUNT DISTINCT as part of a subquery") { - val command = genAST("select * from (select count(distinct key) + 10 from src) numDistincts") - val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) - val expectedRewrite = genAST(""" - select * from - (select count(*) + 10 from - (select distinct key from src) %s) numDistincts - """.format(ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "1")) - checkEquals(rewrite, expectedRewrite) - } - - test("COUNT DISTINCT from results of a subquery") { - val command = genAST(""" - select count(distinct a.val) from - (select * from src where key is null) a - join - (select * from src where key is null) b on a.key = b.key - """) - val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) - val expectedRewrite = genAST(""" - select count(*) from - (select distinct a.val from - (select * from src where key is null) a - join - (select * from src where key is null) b on a.key = b.key - ) %s - """.format(ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "0")) - checkEquals(rewrite, expectedRewrite) - } - - test("COUNT DISTINCT from the results of a subquery, as part of an outer subquery") { - val command = genAST(""" - select * from ( - select count(distinct a.val) from - (select * from src where key is null) a - join - (select * from src where key is null) b on a.key = b.key - ) numDistincts - """) - val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) - val expectedRewrite = genAST(""" - select * from - (select count(*) from - (select distinct a.val from - (select * from src where key is null) a - join - (select * from src where key is null) b on a.key = b.key - ) %s - ) numDistincts - """.format(ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "1")) - checkEquals(rewrite, expectedRewrite) - } - - test("Union multiple count distincts") { - val command = genAST(""" - select * from ( - select count(distinct key) from src - union all - select count(distinct value) from src - union all - select count(distinct key) from src1 - union all - select count(distinct value) from src2 - ) distinctKVs""") - val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) - val expectedRewrite = genAST(""" - select * from ( - select count(*) from (select distinct key from src) %s - union all - select count(*) from (select distinct value from src) %s - union all - select count(*) from (select distinct key from src1) %s - union all - select count(*) from (select distinct value from src2) %s - ) distinctKVs""".format( - ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "3", - ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "4", - ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "2", - ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "1")) - checkEquals(rewrite, expectedRewrite) - } - - test("Union multiple count distincts, both over subqueries") { - val command = genAST(""" - select * from ( - select count(distinct a.key) from - (select * from src where key is null) a - join - (select * from src where key is null) b on a.key = b.key - union all - select count(distinct c.value) from - (select * from src where value is null) c - join - (select * from src where value is null) d on c.value = d.value - ) distinctKVs""") - val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) - val expectedRewrite = genAST(""" - select * from ( - select count(*) from - (select distinct a.key from - (select * from src where key is null) a - join - (select * from src where key is null) b on a.key = b.key - ) %s - union all - select count(*) from - (select distinct c.value from - (select * from src where value is null) c - join - (select * from src where value is null) d on c.value = d.value - ) %s - ) distinctKVs""".format( - ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "1", - ASTRewriteUtil.DISTINCT_SUBQUERY_ALIAS + "2")) - checkEquals(rewrite, expectedRewrite) - } - - test("Multiple COUNT DISTINCT in SELECT expression list isn't rewritten (or supported yet)") { - val command = genAST(""" - select - sum(key), - count(distinct key), - count(distinct value) - from src""") - val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) - checkEquals(command, rewrite) - } - - test("COUNT DISTINCT with partitioning key isn't rewritten") { - val command = genAST("select key, count(distinct value) from src group by key") - val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) - checkEquals(command, rewrite) - } - - test("COUNT DISTINCT with LIMIT isn't rewritten") { - val command = genAST("select key, count(distinct value) from src limit 10") - val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) - checkEquals(command, rewrite) - } - - test("COUNT DISTINCT with CUBE and GROUP BY isn't rewritten") { - val command = genAST("select key, count(distinct value) from src group by key with cube") - val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) - checkEquals(command, rewrite) - } - - test("COUNT DISTINCT with ROLLUP and GROUP BY isn't rewritten") { - val command = genAST("select key, count(distinct value) from src group by key with rollup") - val rewrite = ASTRewriteUtil.countDistinctToGroupBy(command) - checkEquals(command, rewrite) - } - -} diff --git a/src/main/test/scala/shark/ReflectionSuite.scala b/src/main/test/scala/shark/ReflectionSuite.scala deleted file mode 100755 index b2445505..00000000 --- a/src/main/test/scala/shark/ReflectionSuite.scala +++ /dev/null @@ -1,112 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark - -import org.scalatest.FunSuite - - -/** - * A suite of test to ensure reflections are used properly in Shark to invoke - * Hive non-public methods. This is needed because we cannot detect reflection - * errors until runtime. Every time reflection is used to expand visibility of - * methods or variables, a test should be added. - */ -class ReflectionSuite extends FunSuite { - - test("CliDriver") { - val c = classOf[org.apache.hadoop.hive.cli.CliDriver] - var m = c.getDeclaredMethod("getFormattedDb", - classOf[org.apache.hadoop.hive.conf.HiveConf], - classOf[org.apache.hadoop.hive.cli.CliSessionState]) - m.setAccessible(true) - assert(m.getReturnType == classOf[String]) - - m = c.getDeclaredMethod( - "spacesForString", classOf[String]) - m.setAccessible(true) - assert(m.getReturnType == classOf[String]) - } - - test("Driver") { - val c = classOf[org.apache.hadoop.hive.ql.Driver] - - var m = c.getDeclaredMethod( - "doAuthorization", classOf[org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer]) - m.setAccessible(true) - assert(m.getReturnType === Void.TYPE) - - m = c.getDeclaredMethod("getHooks", - classOf[org.apache.hadoop.hive.conf.HiveConf.ConfVars], classOf[Class[_]]) - m.setAccessible(true) - assert(m.getReturnType === classOf[java.util.List[_]]) - - var f = c.getDeclaredField("plan") - f.setAccessible(true) - assert(f.getType === classOf[org.apache.hadoop.hive.ql.QueryPlan]) - - f = c.getDeclaredField("ctx") - f.setAccessible(true) - assert(f.getType === classOf[org.apache.hadoop.hive.ql.Context]) - - f = c.getDeclaredField("schema") - f.setAccessible(true) - assert(f.getType === classOf[org.apache.hadoop.hive.metastore.api.Schema]) - - f = c.getDeclaredField("LOG") - f.setAccessible(true) - assert(f.getType === classOf[org.apache.commons.logging.Log]) - } - - test("SemanticAnalyzer") { - val c = classOf[org.apache.hadoop.hive.ql.parse.SemanticAnalyzer] - var m = c.getDeclaredMethod( - "validateCreateView", - classOf[org.apache.hadoop.hive.ql.plan.CreateViewDesc]) - m.setAccessible(true) - assert(m.getReturnType === Void.TYPE) - - m = c.getDeclaredMethod( - "convertRowSchemaToViewSchema", - classOf[org.apache.hadoop.hive.ql.parse.RowResolver]) - m.setAccessible(true) - assert(m.getReturnType === classOf[java.util.List[_]]) - - val f = c.getDeclaredField("viewsExpanded") - f.setAccessible(true) - assert(f.getType === classOf[java.util.ArrayList[_]]) - } - - test("UnionOperator") { - val c = classOf[org.apache.hadoop.hive.ql.exec.UnionOperator] - var f = c.getDeclaredField("needsTransform") - f.setAccessible(true) - assert(f.getType === classOf[Array[Boolean]]) - } - - test("FileSinkOperator") { - val fileSinkCls = classOf[org.apache.hadoop.hive.ql.exec.FileSinkOperator] - var f = fileSinkCls.getDeclaredField("fsp") - f.setAccessible(true) - assert(f.getType === classOf[org.apache.hadoop.hive.ql.exec.FileSinkOperator#FSPaths]) - - val fspCls = classOf[org.apache.hadoop.hive.ql.exec.FileSinkOperator#FSPaths] - f = fspCls.getDeclaredField("finalPaths") - f.setAccessible(true) - assert(f.getType === classOf[Array[org.apache.hadoop.fs.Path]]) - } -} diff --git a/src/main/test/scala/shark/SQLSuite.scala b/src/main/test/scala/shark/SQLSuite.scala deleted file mode 100755 index 8175c308..00000000 --- a/src/main/test/scala/shark/SQLSuite.scala +++ /dev/null @@ -1,1154 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark - -import scala.collection.JavaConversions._ - -import org.scalatest.FunSuite - -import org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME -import org.apache.hadoop.hive.ql.metadata.Hive -import org.apache.spark.rdd.RDD -import org.apache.spark.rdd.UnionRDD -import org.apache.spark.storage.StorageLevel - -import shark.api.QueryExecutionException -import shark.memstore2.{CacheType, MemoryMetadataManager, PartitionedMemoryTable} -import shark.tgf.{RDDSchema, Schema} -// import expectSql() shortcut methods -import shark.SharkRunner._ - - -class SQLSuite extends FunSuite { - - val DEFAULT_DB_NAME = DEFAULT_DATABASE_NAME - val KV1_TXT_PATH = "${hiveconf:shark.test.data.path}/kv1.txt" - - var sc: SharkContext = SharkRunner.init() - var sharkMetastore: MemoryMetadataManager = SharkEnv.memoryMetadataManager - - private def createCachedPartitionedTable( - tableName: String, - numPartitionsToCreate: Int, - maxCacheSize: Int = 10, - cachePolicyClassName: String = "shark.memstore2.LRUCachePolicy" - ): PartitionedMemoryTable = { - sc.runSql("drop table if exists %s".format(tableName)) - sc.runSql(""" - create table %s(key int, value string) - partitioned by (keypart int) - tblproperties('shark.cache' = 'true', - 'shark.cache.policy.maxSize' = '%d', - 'shark.cache.policy' = '%s') - """.format( - tableName, - maxCacheSize, - cachePolicyClassName)) - var partitionNum = 1 - while (partitionNum <= numPartitionsToCreate) { - sc.runSql("""insert into table %s partition(keypart = %d) - select * from test_cached""".format(tableName, partitionNum)) - partitionNum += 1 - } - assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( - DEFAULT_DB_NAME, tableName).get - partitionedTable - } - - def isFlattenedUnionRDD(unionRDD: UnionRDD[_]) = { - unionRDD.rdds.find(_.isInstanceOf[UnionRDD[_]]).isEmpty - } - - // Takes a sum over the table's 'key' column, for both the cached contents and the copy on disk. - def expectUnifiedKVTable( - cachedTableName: String, - partSpecOpt: Option[Map[String, String]] = None) { - // Check that the table is in memory and is a unified view. - val sharkTableOpt = sharkMetastore.getTable(DEFAULT_DB_NAME, cachedTableName) - assert(sharkTableOpt.isDefined, "Table %s cannot be found in the Shark metastore") - assert(sharkTableOpt.get.cacheMode == CacheType.MEMORY, - "'shark.cache' field for table %s is not CacheType.MEMORY") - - // Load a non-cached copy of the table into memory. - val cacheSum = sc.sql("select sum(key) from %s".format(cachedTableName))(0) - val hiveTable = Hive.get().getTable(DEFAULT_DB_NAME, cachedTableName) - val location = partSpecOpt match { - case Some(partSpec) => { - val partition = Hive.get().getPartition(hiveTable, partSpec, false /* forceCreate */) - partition.getDataLocation.toString - } - case None => hiveTable.getDataLocation.toString - } - // Create a table with contents loaded from the table's data directory. - val diskTableName = "%s_disk_copy".format(cachedTableName) - sc.sql("drop table if exists %s".format(diskTableName)) - sc.sql("create table %s (key int, value string)".format(diskTableName)) - sc.sql("load data local inpath '%s' into table %s".format(location, diskTableName)) - val diskSum = sc.sql("select sum(key) from %s".format(diskTableName))(0) - assert(diskSum == cacheSum, "Sum of keys from cached and disk contents differ") - } - - ////////////////////////////////////////////////////////////////////////////// - // basic SQL - ////////////////////////////////////////////////////////////////////////////// - test("count") { - expectSql("select count(*) from test", "500") - expectSql("select count(*) from test_cached", "500") - } - - test("filter") { - expectSql("select * from test where key=100 or key=497", - Array("100\tval_100", "100\tval_100", "497\tval_497")) - expectSql("select * from test_cached where key=100 or key=497", - Array("100\tval_100", "100\tval_100", "497\tval_497")) - } - - test("count distinct") { - sc.runSql("set mapred.reduce.tasks=3") - expectSql("select count(distinct key) from test", "309") - expectSql("select count(distinct key) from test_cached", "309") - expectSql( - """|SELECT substr(key,1,1), count(DISTINCT substr(val,5)) from test - |GROUP BY substr(key,1,1)""".stripMargin, - Array("0\t1", "1\t71", "2\t69", "3\t62", "4\t74", "5\t6", "6\t5", "7\t6", "8\t8", "9\t7")) - } - - test("count bigint") { - sc.runSql("drop table if exists test_bigint") - sc.runSql("create table test_bigint (key bigint, val string)") - sc.runSql("""load data local inpath '${hiveconf:shark.test.data.path}/kv1.txt' - OVERWRITE INTO TABLE test_bigint""") - sc.runSql("drop table if exists test_bigint_cached") - sc.runSql("create table test_bigint_cached as select * from test_bigint") - expectSql("select val, count(*) from test_bigint_cached where key=484 group by val", - "val_484\t1") - } - - test("limit") { - assert(sc.runSql("select * from test limit 10").results.length === 10) - assert(sc.runSql("select * from test limit 501").results.length === 500) - sc.runSql("drop table if exists test_limit0") - assert(sc.runSql("select * from test limit 0").results.length === 0) - assert(sc.runSql("create table test_limit0 as select * from test limit 0").results.length === 0) - assert(sc.runSql("select * from test_limit0 limit 0").results.length === 0) - assert(sc.runSql("select * from test_limit0 limit 1").results.length === 0) - } - - ////////////////////////////////////////////////////////////////////////////// - // sorting - ////////////////////////////////////////////////////////////////////////////// - - ignore("full order by") { - expectSql("select * from users order by id", Array("1\tA", "2\tB", "3\tA"), sort = false) - expectSql("select * from users order by id desc", Array("3\tA", "2\tB", "1\tA"), sort = false) - expectSql("select * from users order by name, id", Array("1\tA", "3\tA", "2\tB"), sort = false) - expectSql("select * from users order by name desc, id desc", Array("2\tB", "3\tA", "1\tA"), - sort = false) - } - - test("full order by with limit") { - expectSql("select * from users order by id limit 2", Array("1\tA", "2\tB"), sort = false) - expectSql("select * from users order by id desc limit 2", Array("3\tA", "2\tB"), sort = false) - expectSql("select * from users order by name, id limit 2", Array("1\tA", "3\tA"), sort = false) - expectSql("select * from users order by name desc, id desc limit 2", Array("2\tB", "3\tA"), - sort = false) - } - - ////////////////////////////////////////////////////////////////////////////// - // join - ////////////////////////////////////////////////////////////////////////////// - test("join ouput rows of stand objects") { - assert( - sc.sql("select test1val from users join test1 on users.id=test1.id and users.id=1").head === - "[0,1,2]") - } - - ////////////////////////////////////////////////////////////////////////////// - // map join - ////////////////////////////////////////////////////////////////////////////// - test("map join") { - expectSql("""select u.name, count(c.click) from clicks c join users u on (c.id = u.id) - group by u.name having u.name='A'""", - "A\t3") - } - - test("map join2") { - expectSql("select count(*) from clicks join users on (clicks.id = users.id)", "5") - } - - ////////////////////////////////////////////////////////////////////////////// - // join - ////////////////////////////////////////////////////////////////////////////// - test("outer join on null key") { - expectSql("""select count(distinct a.val) from - (select * from test_null where key is null) a - left outer join - (select * from test_null where key is null) b on a.key=b.key""", "7") - } - - ////////////////////////////////////////////////////////////////////////////// - // cache DDL - ////////////////////////////////////////////////////////////////////////////// - test("Use regular CREATE TABLE and '_cached' suffix to create cached table") { - sc.runSql("drop table if exists empty_table_cached") - sc.runSql("create table empty_table_cached(key string, value string)") - assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, "empty_table_cached")) - assert(!SharkEnv.memoryMetadataManager.isHivePartitioned(DEFAULT_DB_NAME, "empty_table_cached")) - } - - test("Use regular CREATE TABLE and table properties to create cached table") { - sc.runSql("drop table if exists empty_table_cached_tbl_props") - sc.runSql("""create table empty_table_cached_tbl_props(key string, value string) - TBLPROPERTIES('shark.cache' = 'true')""") - assert(SharkEnv.memoryMetadataManager.containsTable( - DEFAULT_DB_NAME, "empty_table_cached_tbl_props")) - assert(!SharkEnv.memoryMetadataManager.isHivePartitioned( - DEFAULT_DB_NAME, "empty_table_cached_tbl_props")) - } - - test("Insert into empty cached table") { - sc.runSql("drop table if exists new_table_cached") - sc.runSql("create table new_table_cached(key string, value string)") - sc.runSql("insert into table new_table_cached select * from test where key > -1 limit 499") - expectSql("select count(*) from new_table_cached", "499") - } - - test("rename cached table") { - sc.runSql("drop table if exists test_oldname_cached") - sc.runSql("drop table if exists test_rename") - sc.runSql("create table test_oldname_cached as select * from test") - sc.runSql("alter table test_oldname_cached rename to test_rename") - - assert(!SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, "test_oldname_cached")) - assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, "test_rename")) - - expectSql("select count(*) from test_rename", "500") - } - - test("insert into cached tables") { - sc.runSql("drop table if exists test1_cached") - sc.runSql("create table test1_cached as select * from test") - expectSql("select count(*) from test1_cached", "500") - sc.runSql("insert into table test1_cached select * from test where key > -1 limit 499") - expectSql("select count(*) from test1_cached", "999") - } - - test("insert overwrite") { - sc.runSql("drop table if exists test2_cached") - sc.runSql("create table test2_cached as select * from test") - expectSql("select count(*) from test2_cached", "500") - sc.runSql("insert overwrite table test2_cached select * from test where key > -1 limit 499") - expectSql("select count(*) from test2_cached", "499") - } - - test("error when attempting to update cached table(s) using command with multiple INSERTs") { - sc.runSql("drop table if exists multi_insert_test") - sc.runSql("drop table if exists multi_insert_test_cached") - sc.runSql("create table multi_insert_test as select * from test") - sc.runSql("create table multi_insert_test_cached as select * from test") - intercept[QueryExecutionException] { - sc.runSql("""from test - insert into table multi_insert_test select * - insert into table multi_insert_test_cached select *""") - } - } - - test("create cached table with 'shark.cache' flag in table properties") { - sc.runSql("drop table if exists ctas_tbl_props") - sc.runSql("""create table ctas_tbl_props TBLPROPERTIES ('shark.cache'='true') as - select * from test""") - assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, "ctas_tbl_props")) - expectSql("select * from ctas_tbl_props where key=407", "407\tval_407") - } - - test("default to Hive table creation when 'shark.cache' flag is false in table properties") { - sc.runSql("drop table if exists ctas_tbl_props_should_not_be_cached") - sc.runSql(""" - CREATE TABLE ctas_tbl_props_result_should_not_be_cached - TBLPROPERTIES ('shark.cache'='false') - AS select * from test""") - assert(!SharkEnv.memoryMetadataManager.containsTable( - DEFAULT_DB_NAME, "ctas_tbl_props_should_not_be_cached")) - } - - test("cached tables with complex types") { - sc.runSql("drop table if exists test_complex_types") - sc.runSql("drop table if exists test_complex_types_cached") - sc.runSql("""CREATE TABLE test_complex_types ( - a STRING, b ARRAY, c ARRAY>, d MAP>)""") - sc.runSql("""load data local inpath '${hiveconf:shark.test.data.path}/create_nested_type.txt' - overwrite into table test_complex_types""") - sc.runSql("""create table test_complex_types_cached TBLPROPERTIES ("shark.cache" = "true") as - select * from test_complex_types""") - - assert(sc.sql("select a from test_complex_types_cached where a = 'a0'").head === "a0") - - assert(sc.sql("select b from test_complex_types_cached where a = 'a0'").head === - """["b00","b01"]""") - - assert(sc.sql("select c from test_complex_types_cached where a = 'a0'").head === - """[{"c001":"C001","c002":"C002"},{"c011":null,"c012":"C012"}]""") - - assert(sc.sql("select d from test_complex_types_cached where a = 'a0'").head === - """{"d01":["d011","d012"],"d02":["d021","d022"]}""") - - assert(SharkEnv.memoryMetadataManager.containsTable( - DEFAULT_DB_NAME, "test_complex_types_cached")) - } - - test("disable caching by default") { - sc.runSql("set shark.cache.flag.checkTableName=false") - sc.runSql("drop table if exists should_not_be_cached") - sc.runSql("create table should_not_be_cached as select * from test") - expectSql("select key from should_not_be_cached where key = 407", "407") - assert(!SharkEnv.memoryMetadataManager.containsTable( - DEFAULT_DB_NAME, "should_not_be_cached")) - sc.runSql("set shark.cache.flag.checkTableName=true") - } - - test("cached table name should be case-insensitive") { - sc.runSql("drop table if exists sharkTest5Cached") - sc.runSql("""create table sharkTest5Cached TBLPROPERTIES ("shark.cache" = "true") as - select * from test""") - expectSql("select val from sharktest5Cached where key = 407", "val_407") - assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, "sharkTest5Cached")) - } - - test("dropping cached tables should clean up RDDs") { - sc.runSql("drop table if exists sharkTest5Cached") - sc.runSql("""create table sharkTest5Cached TBLPROPERTIES ("shark.cache" = "true") as - select * from test""") - sc.runSql("drop table sharkTest5Cached") - assert(!SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, "sharkTest5Cached")) - } - - test("lateral view explode column pruning") { - // If column pruner doesn't take lateral view into account, the first result will be null. - assert(sc.runSql("""select * from test_cached - lateral view explode(array(1, 2, 3)) exploadedTbl as col1""").results.head.head != null) - } - - ////////////////////////////////////////////////////////////////////////////// - // Caching Hive-partititioned tables - // Note: references to 'partition' for this section refer to a Hive-partition. - ////////////////////////////////////////////////////////////////////////////// - test("Use regular CREATE TABLE and '_cached' suffix to create cached, partitioned table") { - sc.runSql("drop table if exists empty_part_table_cached") - sc.runSql("""create table empty_part_table_cached(key int, value string) - partitioned by (keypart int)""") - assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, "empty_part_table_cached")) - assert(SharkEnv.memoryMetadataManager.isHivePartitioned( - DEFAULT_DB_NAME, "empty_part_table_cached")) - } - - test("Use regular CREATE TABLE and table properties to create cached, partitioned table") { - sc.runSql("drop table if exists empty_part_table_cached_tbl_props") - sc.runSql("""create table empty_part_table_cached_tbl_props(key int, value string) - partitioned by (keypart int) tblproperties('shark.cache' = 'true')""") - assert(SharkEnv.memoryMetadataManager.containsTable( - DEFAULT_DB_NAME, "empty_part_table_cached_tbl_props")) - assert(SharkEnv.memoryMetadataManager.isHivePartitioned( - DEFAULT_DB_NAME, "empty_part_table_cached_tbl_props")) - } - - test("alter cached table by adding a new partition") { - sc.runSql("drop table if exists alter_part_cached") - sc.runSql("""create table alter_part_cached(key int, value string) - partitioned by (keypart int)""") - sc.runSql("""alter table alter_part_cached add partition(keypart = 1)""") - val tableName = "alter_part_cached" - val partitionColumn = "keypart=1" - assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( - DEFAULT_DB_NAME, tableName).get - assert(partitionedTable.containsPartition(partitionColumn)) - } - - test("alter cached table by dropping a partition") { - sc.runSql("drop table if exists alter_drop_part_cached") - sc.runSql("""create table alter_drop_part_cached(key int, value string) - partitioned by (keypart int)""") - sc.runSql("""alter table alter_drop_part_cached add partition(keypart = 1)""") - val tableName = "alter_drop_part_cached" - val partitionColumn = "keypart=1" - assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( - DEFAULT_DB_NAME, tableName).get - assert(partitionedTable.containsPartition(partitionColumn)) - sc.runSql("""alter table alter_drop_part_cached drop partition(keypart = 1)""") - assert(!partitionedTable.containsPartition(partitionColumn)) - } - - test("insert into a partition of a cached table") { - val tableName = "insert_part_cached" - val partitionedTable = createCachedPartitionedTable( - tableName, - 1 /* numPartitionsToCreate */) - expectSql("select value from insert_part_cached where key = 407 and keypart = 1", "val_407") - - } - - test("insert overwrite a partition of a cached table") { - val tableName = "insert_over_part_cached" - val partitionedTable = createCachedPartitionedTable( - tableName, - 1 /* numPartitionsToCreate */) - expectSql("""select value from insert_over_part_cached - where key = 407 and keypart = 1""", "val_407") - sc.runSql("""insert overwrite table insert_over_part_cached partition(keypart = 1) - select key, -1 from test""") - expectSql("select value from insert_over_part_cached where key = 407 and keypart = 1", "-1") - } - - test("scan cached, partitioned table that's empty") { - sc.runSql("drop table if exists empty_part_table_cached") - sc.runSql("""create table empty_part_table_cached(key int, value string) - partitioned by (keypart int)""") - expectSql("select count(*) from empty_part_table_cached", "0") - } - - test("scan cached, partitioned table that has a single partition") { - val tableName = "scan_single_part_cached" - val partitionedTable = createCachedPartitionedTable( - tableName, - 1 /* numPartitionsToCreate */) - expectSql("select * from scan_single_part_cached where key = 407", "407\tval_407\t1") - } - - test("scan cached, partitioned table that has multiple partitions") { - val tableName = "scan_mult_part_cached" - val partitionedTable = createCachedPartitionedTable( - tableName, - 3 /* numPartitionsToCreate */) - expectSql("select * from scan_mult_part_cached where key = 407 order by keypart", - Array("407\tval_407\t1", "407\tval_407\t2", "407\tval_407\t3")) - } - - test("drop/unpersist cached, partitioned table that has multiple partitions") { - val tableName = "drop_mult_part_cached" - val partitionedTable = createCachedPartitionedTable( - tableName, - 3 /* numPartitionsToCreate */) - val keypart1RDD = partitionedTable.getPartition("keypart=1") - val keypart2RDD = partitionedTable.getPartition("keypart=2") - val keypart3RDD = partitionedTable.getPartition("keypart=3") - sc.runSql("drop table drop_mult_part_cached ") - assert(!SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) - // All RDDs should have been unpersisted. - assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE) - assert(keypart2RDD.get.getStorageLevel == StorageLevel.NONE) - assert(keypart3RDD.get.getStorageLevel == StorageLevel.NONE) - } - - test("drop cached partition represented by a UnionRDD (i.e., the result of multiple inserts)") { - val tableName = "drop_union_part_cached" - val partitionedTable = createCachedPartitionedTable( - tableName, - 1 /* numPartitionsToCreate */) - sc.runSql("insert into table drop_union_part_cached partition(keypart = 1) select * from test") - sc.runSql("insert into table drop_union_part_cached partition(keypart = 1) select * from test") - sc.runSql("insert into table drop_union_part_cached partition(keypart = 1) select * from test") - val keypart1RDD = partitionedTable.getPartition("keypart=1") - sc.runSql("drop table drop_union_part_cached") - assert(!SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) - // All RDDs should have been unpersisted. - assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE) - } - - ////////////////////////////////////////////////////////////////////////////// - // RDD(partition) eviction policy for cached Hive-partititioned tables - ////////////////////////////////////////////////////////////////////////////// - - test("shark.memstore2.CacheAllPolicy is the default policy") { - val tableName = "default_policy_cached" - sc.runSql("""create table default_policy_cached(key int, value string) - partitioned by (keypart int)""") - assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( - DEFAULT_DB_NAME, tableName).get - val cachePolicy = partitionedTable.cachePolicy - assert(cachePolicy.isInstanceOf[shark.memstore2.CacheAllPolicy[_, _]]) - } - - test("LRU: RDDs are not evicted if the cache isn't full.") { - val tableName = "evict_partitions_maxSize" - val partitionedTable = createCachedPartitionedTable( - tableName, - 2 /* numPartitionsToCreate */, - 3 /* maxCacheSize */, - "shark.memstore2.LRUCachePolicy") - val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") - assert(TestUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) - } - - test("LRU: RDDs are evicted when the max size is reached.") { - val tableName = "evict_partitions_maxSize" - val partitionedTable = createCachedPartitionedTable( - tableName, - 3 /* numPartitionsToCreate */, - 3 /* maxCacheSize */, - "shark.memstore2.LRUCachePolicy") - val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") - assert(TestUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) - sc.runSql("""insert into table evict_partitions_maxSize partition(keypart = 4) - select * from test""") - assert(TestUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.NONE) - } - - test("LRU: RDD eviction accounts for partition scans - a cache.get()") { - val tableName = "evict_partitions_with_get" - val partitionedTable = createCachedPartitionedTable( - tableName, - 3 /* numPartitionsToCreate */, - 3 /* maxCacheSize */, - "shark.memstore2.LRUCachePolicy") - val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") - val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") - assert(TestUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) - assert(TestUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) - sc.runSql("select count(1) from evict_partitions_with_get where keypart = 1") - sc.runSql("""insert into table evict_partitions_with_get partition(keypart = 4) - select * from test""") - assert(TestUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) - - assert(TestUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.NONE) - } - - test("LRU: RDD eviction accounts for INSERT INTO - a cache.get().") { - val tableName = "evict_partitions_insert_into" - val partitionedTable = createCachedPartitionedTable( - tableName, - 3 /* numPartitionsToCreate */, - 3 /* maxCacheSize */, - "shark.memstore2.LRUCachePolicy") - assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) - val oldKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") - val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") - assert(TestUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) - assert(TestUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) - sc.runSql("""insert into table evict_partitions_insert_into partition(keypart = 1) - select * from test""") - sc.runSql("""insert into table evict_partitions_insert_into partition(keypart = 4) - select * from test""") - assert(TestUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) - val newKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") - assert(TestUtils.getStorageLevelOfRDD(newKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) - - val keypart2StorageLevel = TestUtils.getStorageLevelOfRDD(keypart2RDD.get) - assert(keypart2StorageLevel == StorageLevel.NONE) - } - - test("LRU: RDD eviction accounts for INSERT OVERWRITE - a cache.put()") { - val tableName = "evict_partitions_insert_overwrite" - val partitionedTable = createCachedPartitionedTable( - tableName, - 3 /* numPartitionsToCreate */, - 3 /* maxCacheSize */, - "shark.memstore2.LRUCachePolicy") - assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) - val oldKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") - val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") - assert(TestUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) - assert(TestUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.MEMORY_AND_DISK) - sc.runSql("""insert overwrite table evict_partitions_insert_overwrite partition(keypart = 1) - select * from test""") - sc.runSql("""insert into table evict_partitions_insert_overwrite partition(keypart = 4) - select * from test""") - assert(TestUtils.getStorageLevelOfRDD(oldKeypart1RDD.get) == StorageLevel.NONE) - val newKeypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") - assert(TestUtils.getStorageLevelOfRDD(newKeypart1RDD.get) == StorageLevel.MEMORY_AND_DISK) - - val keypart2StorageLevel = TestUtils.getStorageLevelOfRDD(keypart2RDD.get) - assert(keypart2StorageLevel == StorageLevel.NONE) - } - - test("LRU: RDD eviction accounts for ALTER TABLE DROP PARTITION - a cache.remove()") { - val tableName = "evict_partitions_removals" - val partitionedTable = createCachedPartitionedTable( - tableName, - 3 /* numPartitionsToCreate */, - 3 /* maxCacheSize */, - "shark.memstore2.LRUCachePolicy") - assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) - sc.runSql("alter table evict_partitions_removals drop partition(keypart = 1)") - sc.runSql("""insert into table evict_partitions_removals partition(keypart = 4) - select * from test""") - sc.runSql("""insert into table evict_partitions_removals partition(keypart = 5) - select * from test""") - val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") - assert(TestUtils.getStorageLevelOfRDD(keypart2RDD.get) == StorageLevel.NONE) - } - - test("LRU: get() reloads an RDD previously unpersist()'d.") { - val tableName = "reload_evicted_partition" - val partitionedTable = createCachedPartitionedTable( - tableName, - 3 /* numPartitionsToCreate */, - 3 /* maxCacheSize */, - "shark.memstore2.LRUCachePolicy") - assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) - val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") - val lvl = TestUtils.getStorageLevelOfRDD(keypart1RDD.get) - assert(lvl == StorageLevel.MEMORY_AND_DISK, "got: " + lvl) - sc.runSql("""insert into table reload_evicted_partition partition(keypart = 4) - select * from test""") - assert(TestUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.NONE) - - // Scanning partition (keypart = 1) should reload the corresponding RDD into the cache, and - // cause eviction of the RDD for partition (keypart = 2). - sc.runSql("select count(1) from reload_evicted_partition where keypart = 1") - assert(keypart1RDD.get.getStorageLevel == StorageLevel.MEMORY_AND_DISK) - val keypart2RDD = partitionedTable.keyToPartitions.get("keypart=2") - val keypart2StorageLevel = TestUtils.getStorageLevelOfRDD(keypart2RDD.get) - assert(keypart2StorageLevel == StorageLevel.NONE, - "StorageLevel for partition(keypart=2) should be NONE, but got: " + keypart2StorageLevel) - } - - test("FIFO: get() does not reload an RDD previously unpersist()'d.") { - val tableName = "dont_reload_evicted_partition" - val partitionedTable = createCachedPartitionedTable( - tableName, - 3 /* numPartitionsToCreate */, - 3 /* maxCacheSize */, - "shark.memstore2.FIFOCachePolicy") - assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, tableName)) - val keypart1RDD = partitionedTable.keyToPartitions.get("keypart=1") - val lvl = TestUtils.getStorageLevelOfRDD(keypart1RDD.get) - assert(lvl == StorageLevel.MEMORY_AND_DISK, "got: " + lvl) - sc.runSql("""insert into table dont_reload_evicted_partition partition(keypart = 4) - select * from test""") - assert(TestUtils.getStorageLevelOfRDD(keypart1RDD.get) == StorageLevel.NONE) - - // Scanning partition (keypart = 1) should reload the corresponding RDD into the cache, and - // cause eviction of the RDD for partition (keypart = 2). - sc.runSql("select count(1) from dont_reload_evicted_partition where keypart = 1") - assert(keypart1RDD.get.getStorageLevel == StorageLevel.NONE, "got: " + keypart1RDD.get.getStorageLevel) - } - - /////////////////////////////////////////////////////////////////////////////////////// - // Prevent nested UnionRDDs - those should be "flattened" in MemoryStoreSinkOperator. - /////////////////////////////////////////////////////////////////////////////////////// - - test("flatten UnionRDDs") { - sc.sql("create table flat_cached as select * from test_cached") - sc.sql("insert into table flat_cached select * from test") - val tableName = "flat_cached" - var memoryTable = SharkEnv.memoryMetadataManager.getMemoryTable(DEFAULT_DB_NAME, tableName).get - var unionRDD = memoryTable.getRDD.get.asInstanceOf[UnionRDD[_]] - val numParentRDDs = unionRDD.rdds.size - assert(isFlattenedUnionRDD(unionRDD)) - - // Insert another set of query results. The flattening should kick in here. - sc.sql("insert into table flat_cached select * from test") - unionRDD = memoryTable.getRDD.get.asInstanceOf[UnionRDD[_]] - assert(isFlattenedUnionRDD(unionRDD)) - assert(unionRDD.rdds.size == numParentRDDs + 1) - } - - test("flatten UnionRDDs for partitioned tables") { - sc.sql("drop table if exists part_table_cached") - sc.sql("""create table part_table_cached(key int, value string) - partitioned by (keypart int)""") - sc.sql("alter table part_table_cached add partition(keypart = 1)") - sc.sql("insert into table part_table_cached partition(keypart = 1) select * from flat_cached") - val tableName = "part_table_cached" - val partitionKey = "keypart=1" - val partitionedTable = SharkEnv.memoryMetadataManager.getPartitionedTable( - DEFAULT_DB_NAME, tableName).get - var unionRDD = partitionedTable.keyToPartitions.get(partitionKey).get.asInstanceOf[UnionRDD[_]] - val numParentRDDs = unionRDD.rdds.size - assert(isFlattenedUnionRDD(unionRDD)) - - // Insert another set of query results into the same partition. - // The flattening should kick in here. - sc.runSql("insert into table part_table_cached partition(keypart = 1) select * from flat_cached") - unionRDD = partitionedTable.getPartition(partitionKey).get.asInstanceOf[UnionRDD[_]] - assert(isFlattenedUnionRDD(unionRDD)) - assert(unionRDD.rdds.size == numParentRDDs + 1) - } - - ////////////////////////////////////////////////////////////////////////////// - // Tableau bug - ////////////////////////////////////////////////////////////////////////////// - - test("tableau bug / adw") { - sc.sql("drop table if exists adw") - sc.sql("""create table adw TBLPROPERTIES ("shark.cache" = "true") as - select cast(key as int) as k, val from test""") - expectSql("select count(k) from adw where val='val_487' group by 1 having count(1) > 0", "1") - } - - ////////////////////////////////////////////////////////////////////////////// - // Partition pruning - ////////////////////////////////////////////////////////////////////////////// - - test("sel star pruning") { - sc.sql("drop table if exists selstar") - sc.sql("""create table selstar TBLPROPERTIES ("shark.cache" = "true") as - select * from test""") - expectSql("select * from selstar where val='val_487'","487 val_487") - } - - test("map pruning with functions in between clause") { - sc.sql("drop table if exists mapsplitfunc") - sc.sql("drop table if exists mapsplitfunc_cached") - sc.sql("create table mapsplitfunc(k bigint, v string)") - sc.sql("""load data local inpath '${hiveconf:shark.test.data.path}/kv1.txt' - OVERWRITE INTO TABLE mapsplitfunc""") - sc.sql("create table mapsplitfunc_cached as select * from mapsplitfunc") - expectSql("""select count(*) from mapsplitfunc_cached - where month(from_unixtime(k)) between "1" and "12" """, Array[String]("500")) - expectSql("""select count(*) from mapsplitfunc_cached - where year(from_unixtime(k)) between "2013" and "2014" """, Array[String]("0")) - } - - test("map pruning with functions in in clause") { - expectSql("""select count(*) from mapsplitfunc_cached - where year(from_unixtime(k)) in ("2013", concat("201", "4")) """, Array[String]("0")) - } - - ////////////////////////////////////////////////////////////////////////////// - // SharkContext APIs (e.g. sql2rdd, sql) - ////////////////////////////////////////////////////////////////////////////// - - test("cached table in different new database") { - sc.sql("drop table if exists selstar") - sc.sql("""create table selstar TBLPROPERTIES ("shark.cache" = "true") as - select * from default.test """) - sc.sql("use seconddb") - sc.sql("drop table if exists selstar") - sc.sql("""create table selstar TBLPROPERTIES ("shark.cache" = "true") as - select * from default.test where key != 'val_487' """) - - sc.sql("use default") - expectSql("select * from selstar where val='val_487'","487 val_487") - - assert(SharkEnv.memoryMetadataManager.containsTable(DEFAULT_DB_NAME, "selstar")) - assert(SharkEnv.memoryMetadataManager.containsTable("seconddb", "selstar")) - - } - - ////////////////////////////////////////////////////////////////////////////// - // various data types - ////////////////////////////////////////////////////////////////////////////// - - test("boolean data type") { - sc.sql("drop table if exists checkboolean") - sc.sql("""create table checkboolean TBLPROPERTIES ("shark.cache" = "true") as - select key, val, true as flag from test where key < "300" """) - sc.sql("""insert into table checkboolean - select key, val, false as flag from test where key > "300" """) - expectSql("select flag, count(*) from checkboolean group by flag order by flag asc", - Array[String]("false\t208", "true\t292")) - } - - test("byte data type") { - sc.sql("drop table if exists checkbyte") - sc.sql("drop table if exists checkbyte_cached") - sc.sql("""create table checkbyte (key string, val string, flag tinyint) """) - sc.sql("""insert into table checkbyte - select key, val, 1 from test where key < "300" """) - sc.sql("""insert into table checkbyte - select key, val, 0 from test where key > "300" """) - sc.sql("""create table checkbyte_cached as select * from checkbyte""") - expectSql("select flag, count(*) from checkbyte_cached group by flag order by flag asc", - Array[String]("0\t208", "1\t292")) - } - - test("binary data type") { - - sc.sql("drop table if exists checkbinary") - sc.sql("drop table if exists checkbinary_cached") - sc.sql("""create table checkbinary (key string, flag binary) """) - sc.sql("""insert into table checkbinary - select key, cast(val as binary) as flag from test where key < "300" """) - sc.sql("""insert into table checkbinary - select key, cast(val as binary) as flag from test where key > "300" """) - sc.sql("create table checkbinary_cached as select key, flag from checkbinary") - expectSql("select cast(flag as string) as f from checkbinary_cached order by f asc limit 2", - Array[String]("val_0", "val_0")) - } - - test("short data type") { - sc.sql("drop table if exists checkshort") - sc.sql("drop table if exists checkshort_cached") - sc.sql("""create table checkshort (key string, val string, flag smallint) """) - sc.sql("""insert into table checkshort - select key, val, 23 as flag from test where key < "300" """) - sc.sql("""insert into table checkshort - select key, val, 36 as flag from test where key > "300" """) - sc.sql("create table checkshort_cached as select key, val, flag from checkshort") - expectSql("select flag, count(*) from checkshort_cached group by flag order by flag asc", - Array[String]("23\t292", "36\t208")) - } - - ////////////////////////////////////////////////////////////////////////////// - // SharkContext APIs (e.g. sql2rdd, sql) - ////////////////////////////////////////////////////////////////////////////// - - test("sql max number of rows returned") { - assert(sc.runSql("select * from test").results.size === 500) - assert(sc.runSql("select * from test", 100).results.size === 100) - } - - test("sql2rdd") { - var rdd = sc.sql2rdd("select * from test") - assert(rdd.count === 500) - rdd = sc.sql2rdd("select * from test_cached") - assert(rdd.count === 500) - val collected = rdd.map(r => r.getInt(0)).collect().sortWith(_ < _) - assert(collected(0) === 0) - assert(collected(499) === 498) - assert(collected.size === 500) - } - - test("null values in sql2rdd") { - val nullsRdd = sc.sql2rdd("select * from test_null where key is null") - val nulls = nullsRdd.map(r => r.getInt(0)).collect() - assert(nulls(0) === null) - assert(nulls.size === 10) - } - - test("sql exception") { - val e = intercept[QueryExecutionException] { sc.runSql("asdfasdfasdfasdf") } - e.getMessage.contains("semantic") - } - - test("sql2rdd exception") { - val e = intercept[QueryExecutionException] { sc.sql2rdd("asdfasdfasdfasdf") } - e.getMessage.contains("semantic") - } - - ////////////////////////////////////////////////////////////////////////////// - // Default cache mode is CacheType.MEMORY (unified view) - ////////////////////////////////////////////////////////////////////////////// - test ("Table created by CREATE TABLE, with table properties, is CacheType.MEMORY by default") { - sc.runSql("drop table if exists test_unify_creation") - sc.runSql("""create table test_unify_creation (key int, val string) - tblproperties('shark.cache'='true')""") - val table = sharkMetastore.getTable(DEFAULT_DB_NAME, "test_unify_creation").get - assert(table.cacheMode == CacheType.MEMORY) - sc.runSql("drop table if exists test_unify_creation") - } - - test ("Table created by CREATE TABLE, with '_cached', is CacheType.MEMORY_ONLY by default") { - sc.runSql("drop table if exists test_unify_creation_cached") - sc.runSql("create table test_unify_creation_cached(key int, val string)") - val table = sharkMetastore.getTable(DEFAULT_DB_NAME, "test_unify_creation_cached").get - assert(table.cacheMode == CacheType.MEMORY_ONLY) - sc.runSql("drop table if exists test_unify_creation_cached") - } - - test ("Table created by CTAS, with table properties, is CacheType.MEMORY by default") { - sc.runSql("drop table if exists test_unify_ctas") - sc.runSql("""create table test_unify_ctas tblproperties('shark.cache' = 'true') - as select * from test""") - val table = sharkMetastore.getTable(DEFAULT_DB_NAME, "test_unify_ctas").get - assert(table.cacheMode == CacheType.MEMORY) - expectSql("select count(*) from test_unify_ctas", "500") - sc.runSql("drop table if exists test_unify_ctas") - } - - test ("Table created by CTAS, with '_cached', is CacheType.MEMORY_ONLY by default") { - sc.runSql("drop table if exists test_unify_ctas_cached") - sc.runSql("create table test_unify_ctas_cached as select * from test") - val table = sharkMetastore.getTable(DEFAULT_DB_NAME, "test_unify_ctas_cached").get - assert(table.cacheMode == CacheType.MEMORY_ONLY) - expectSql("select count(*) from test_unify_ctas_cached", "500") - sc.runSql("drop table if exists test_unify_ctas_cached") - } - - test ("CREATE TABLE when 'shark.cache' is CacheType.MEMORY_ONLY") { - sc.runSql("drop table if exists test_non_unify_creation") - sc.runSql("""create table test_non_unify_creation(key int, val string) - tblproperties('shark.cache' = 'memory_only')""") - val table = sharkMetastore.getTable(DEFAULT_DB_NAME, "test_non_unify_creation").get - assert(table.cacheMode == CacheType.MEMORY_ONLY) - sc.runSql("drop table if exists test_non_unify_creation") - } - - test ("CTAS when 'shark.cache' is CacheType.MEMORY_ONLY") { - sc.runSql("drop table if exists test_non_unify_ctas") - sc.runSql("""create table test_non_unify_ctas tblproperties - ('shark.cache' = 'memory_only') as select * from test""") - val table = sharkMetastore.getTable(DEFAULT_DB_NAME, "test_non_unify_ctas").get - assert(table.cacheMode == CacheType.MEMORY_ONLY) - sc.runSql("drop table if exists test_non_unify_ctas") - } - - ////////////////////////////////////////////////////////////////////////////// - // LOAD for tables cached in memory and stored on disk (unified view) - ////////////////////////////////////////////////////////////////////////////// - test ("LOAD INTO unified view") { - sc.runSql("drop table if exists unified_view_cached") - sc.runSql( - """create table unified_view_cached (key int, value string) - |tblproperties("shark.cache" = "memory") - """.stripMargin) - sc.runSql("load data local inpath '%s' into table unified_view_cached".format(KV1_TXT_PATH)) - expectUnifiedKVTable("unified_view_cached") - expectSql("select count(*) from unified_view_cached", "500") - sc.runSql("drop table if exists unified_view_cached") - } - - test ("LOAD OVERWRITE unified view") { - sc.runSql("drop table if exists unified_overwrite_cached") - sc.runSql("create table unified_overwrite_cached (key int, value string)" + - "tblproperties(\"shark.cache\" = \"memory\")") - sc.runSql("load data local inpath '%s' into table unified_overwrite_cached". - format("${hiveconf:shark.test.data.path}/kv3.txt")) - expectSql("select count(*) from unified_overwrite_cached", "25") - sc.runSql("load data local inpath '%s' overwrite into table unified_overwrite_cached". - format(KV1_TXT_PATH)) - // Make sure the cached contents matches the disk contents. - expectUnifiedKVTable("unified_overwrite_cached") - expectSql("select count(*) from unified_overwrite_cached", "500") - sc.runSql("drop table if exists unified_overwrite_cached") - } - - test ("LOAD INTO partitioned unified view") { - sc.runSql("drop table if exists unified_view_part_cached") - sc.runSql("""create table unified_view_part_cached (key int, value string) - partitioned by (keypart int) tblproperties("shark.cache" = "memory")""") - sc.runSql("""load data local inpath '%s' into table unified_view_part_cached - partition(keypart = 1)""".format(KV1_TXT_PATH)) - expectUnifiedKVTable("unified_view_part_cached", Some(Map("keypart" -> "1"))) - expectSql("select count(*) from unified_view_part_cached", "500") - sc.runSql("drop table if exists unified_view_part_cached") - } - - test ("LOAD OVERWRITE partitioned unified view") { - sc.runSql("drop table if exists unified_overwrite_part_cached") - sc.runSql("""create table unified_overwrite_part_cached (key int, value string) - partitioned by (keypart int) tblproperties("shark.cache" = "memory")""") - sc.runSql("""load data local inpath '%s' overwrite into table unified_overwrite_part_cached - partition(keypart = 1)""".format(KV1_TXT_PATH)) - expectUnifiedKVTable("unified_overwrite_part_cached", Some(Map("keypart" -> "1"))) - expectSql("select count(*) from unified_overwrite_part_cached", "500") - sc.runSql("drop table if exists unified_overwrite_part_cached") - } - - ////////////////////////////////////////////////////////////////////////////// - // INSERT for tables cached in memory and stored on disk (unified view) - ////////////////////////////////////////////////////////////////////////////// - test ("INSERT INTO unified view") { - sc.runSql("drop table if exists unified_view_cached") - sc.runSql("create table unified_view_cached tblproperties('shark.cache'='memory') " + - "as select * from test_cached") - sc.runSql("insert into table unified_view_cached select * from test_cached") - expectUnifiedKVTable("unified_view_cached") - expectSql("select count(*) from unified_view_cached", "1000") - sc.runSql("drop table if exists unified_view_cached") - } - - test ("INSERT OVERWRITE unified view") { - sc.runSql("drop table if exists unified_overwrite_cached") - sc.runSql("create table unified_overwrite_cached tblproperties('shark.cache'='memory')" + - "as select * from test") - sc.runSql("insert overwrite table unified_overwrite_cached select * from test_cached") - expectUnifiedKVTable("unified_overwrite_cached") - expectSql("select count(*) from unified_overwrite_cached", "500") - sc.runSql("drop table if exists unified_overwrite_cached") - } - - test ("INSERT INTO partitioned unified view") { - sc.runSql("drop table if exists unified_view_part_cached") - sc.runSql("""create table unified_view_part_cached (key int, value string) - partitioned by (keypart int) - tblproperties('shark.cache'='memory')""") - sc.runSql("""insert into table unified_view_part_cached partition (keypart = 1) - select * from test_cached""") - expectUnifiedKVTable("unified_view_part_cached", Some(Map("keypart" -> "1"))) - expectSql("select count(*) from unified_view_part_cached where keypart = 1", "500") - sc.runSql("drop table if exists unified_view_part_cached") - } - - test ("INSERT OVERWRITE partitioned unified view") { - sc.runSql("drop table if exists unified_overwrite_part_cached") - sc.runSql("""create table unified_overwrite_part_cached (key int, value string) - partitioned by (keypart int) tblproperties('shark.cache'='memory')""") - sc.runSql("""insert overwrite table unified_overwrite_part_cached partition (keypart = 1) - select * from test_cached""") - expectUnifiedKVTable("unified_overwrite_part_cached", Some(Map("keypart" -> "1"))) - expectSql("select count(*) from unified_overwrite_part_cached", "500") - sc.runSql("drop table if exists unified_overwrite_part_cached") - } - - ////////////////////////////////////////////////////////////////////////////// - // CACHE and ALTER TABLE commands - ////////////////////////////////////////////////////////////////////////////// - test ("ALTER TABLE caches non-partitioned table if 'shark.cache' is set to true") { - sc.runSql("drop table if exists unified_load") - sc.runSql("create table unified_load as select * from test") - sc.runSql("alter table unified_load set tblproperties('shark.cache' = 'true')") - expectUnifiedKVTable("unified_load") - sc.runSql("drop table if exists unified_load") - } - - test ("ALTER TABLE caches partitioned table if 'shark.cache' is set to true") { - sc.runSql("drop table if exists unified_part_load") - sc.runSql("create table unified_part_load (key int, value string) partitioned by (keypart int)") - sc.runSql("insert into table unified_part_load partition (keypart=1) select * from test_cached") - sc.runSql("alter table unified_part_load set tblproperties('shark.cache' = 'true')") - expectUnifiedKVTable("unified_part_load", Some(Map("keypart" -> "1"))) - sc.runSql("drop table if exists unified_part_load") - } - - test ("ALTER TABLE uncaches non-partitioned table if 'shark.cache' is set to false") { - sc.runSql("drop table if exists unified_load") - sc.runSql("create table unified_load as select * from test") - sc.runSql("alter table unified_load set tblproperties('shark.cache' = 'false')") - assert(!sharkMetastore.containsTable(DEFAULT_DB_NAME, "unified_load")) - expectSql("select count(*) from unified_load", "500") - sc.runSql("drop table if exists unified_load") - } - - test ("ALTER TABLE uncaches partitioned table if 'shark.cache' is set to false") { - sc.runSql("drop table if exists unified_part_load") - sc.runSql("create table unified_part_load (key int, value string) partitioned by (keypart int)") - sc.runSql("insert into table unified_part_load partition (keypart=1) select * from test_cached") - sc.runSql("alter table unified_part_load set tblproperties('shark.cache' = 'false')") - assert(!sharkMetastore.containsTable(DEFAULT_DB_NAME, "unified_part_load")) - expectSql("select count(*) from unified_part_load", "500") - sc.runSql("drop table if exists unified_part_load") - } - - test ("UNCACHE behaves like ALTER TABLE SET TBLPROPERTIES ...") { - sc.runSql("drop table if exists unified_load") - sc.runSql("create table unified_load as select * from test") - sc.runSql("cache unified_load") - // Double check the table properties. - val tableName = "unified_load" - val hiveTable = Hive.get().getTable(DEFAULT_DB_NAME, tableName) - assert(hiveTable.getProperty("shark.cache") == "MEMORY") - // Check that the cache and disk contents are synchronized. - expectUnifiedKVTable(tableName) - sc.runSql("drop table if exists unified_load") - } - - test ("CACHE behaves like ALTER TABLE SET TBLPROPERTIES ...") { - sc.runSql("drop table if exists unified_load") - sc.runSql("create table unified_load as select * from test") - sc.runSql("cache unified_load") - // Double check the table properties. - val tableName = "unified_load" - val hiveTable = Hive.get().getTable(DEFAULT_DB_NAME, tableName) - assert(hiveTable.getProperty("shark.cache") == "MEMORY") - // Check that the cache and disk contents are synchronized. - expectUnifiedKVTable(tableName) - sc.runSql("drop table if exists unified_load") - } - - ////////////////////////////////////////////////////////////////////////////// - // Cached table persistence - ////////////////////////////////////////////////////////////////////////////// - ignore ("Cached tables persist across Shark metastore shutdowns.") { - val globalCachedTableNames = Seq("test_cached", "test_null_cached", "clicks_cached", - "users_cached") - - // Number of rows for each cached table. - val cachedTableCounts = new Array[String](globalCachedTableNames.size) - for ((tableName, i) <- globalCachedTableNames.zipWithIndex) { - val hiveTable = Hive.get().getTable(DEFAULT_DB_NAME, tableName) - val cachedCount = sc.sql("select count(*) from %s".format(tableName))(0) - cachedTableCounts(i) = cachedCount - } - sharkMetastore.shutdown() - for ((tableName, i) <- globalCachedTableNames.zipWithIndex) { - val hiveTable = Hive.get().getTable(DEFAULT_DB_NAME, tableName) - // Check that the number of rows from the table on disk remains the same. - val onDiskCount = sc.sql("select count(*) from %s".format(tableName))(0) - val cachedCount = cachedTableCounts(i) - assert(onDiskCount == cachedCount, """Num rows for %s differ across Shark metastore restart. - (rows cached = %s, rows on disk = %s)""".format(tableName, cachedCount, onDiskCount)) - // Check that we're able to materialize a row - i.e., make sure that table scan operator - // doesn't try to use a ColumnarSerDe when scanning contents on disk (for our test tables, - // LazySimpleSerDes should be used). - sc.sql("select * from %s limit 1".format(tableName)) - } - // Finally, reload all tables. - SharkRunner.loadTables() - } - - ////////////////////////////////////////////////////////////////////////////// - // Window Function Support - ////////////////////////////////////////////////////////////////////////////// - test("window function support") { - expectSql("select id,name,count(id) over (partition by name) from users", - Array[String]("1\tA\t2", "3\tA\t2", "2\tB\t1")) - expectSql("select id,name,sum(id) over(partition by name order by id) from users", - Array[String]("1\tA\t1", "3\tA\t4", "2\tB\t2")) - expectSql("select id,name,sum(id) over(partition by name order by id rows between " + - "unbounded preceding and current row) from users", - Array[String]("1\tA\t1", "3\tA\t4", "2\tB\t2")) - expectSql("select id,name,sum(id) over(partition by name order by id rows between " + - "current row and unbounded following) from users", - Array[String]("1\tA\t4", "3\tA\t3", "2\tB\t2")) - expectSql("select id,name,sum(id) over(partition by name order by id rows between " + - "unbounded preceding and unbounded following) from users", - Array[String]("1\tA\t4", "3\tA\t4", "2\tB\t2")) - expectSql("select id,name,lead(id) over(partition by name order by id) from users", - Array[String]("1\tA\t3", "3\tA\tnull", "2\tB\tnull")) - expectSql("select id,name,lag(id) over(partition by name order by id) from users", - Array[String]("1\tA\tnull", "3\tA\t1", "2\tB\tnull")) - expectSql("select id, name, sum(id) over w1 as sum_id, max(id) over w1 as max_id from users" + - " window w1 as (partition by name)", - Array[String]("2\tB\t2\t2","1\tA\t4\t3","3\tA\t4\t3")) - } - - ////////////////////////////////////////////////////////////////////////////// - // Table Generating Functions (TGFs) - ////////////////////////////////////////////////////////////////////////////// - - test("Simple TGFs") { - expectSql("generate shark.TestTGF1(test, 15)", Array(15,15,15,17,19).map(_.toString).toArray) - } - - test("Saving simple TGFs") { - sc.sql("drop table if exists TGFTestTable") - sc.runSql("generate shark.TestTGF1(test, 15) as TGFTestTable") - expectSql("select * from TGFTestTable", Array(15,15,15,17,19).map(_.toString).toArray) - sc.sql("drop table if exists TGFTestTable") - } - - test("Advanced TGFs") { - expectSql("generate shark.TestTGF2(test, 25)", Array(25,25,25,27,29).map(_.toString).toArray) - } - - test("Saving advanced TGFs") { - sc.sql("drop table if exists TGFTestTable2") - sc.runSql("generate shark.TestTGF2(test, 25) as TGFTestTable2") - expectSql("select * from TGFTestTable2", Array(25,25,25,27,29).map(_.toString).toArray) - sc.sql("drop table if exists TGFTestTable2") - } -} - -object TestTGF1 { - @Schema(spec = "values int") - def apply(test: RDD[(Int, String)], integer: Int) = { - test.map{ case Tuple2(k, v) => Tuple1(k + integer) }.filter{ case Tuple1(v) => v < 20 } - } -} - -object TestTGF2 { - def apply(sc: SharkContext, test: RDD[(Int, String)], integer: Int) = { - val rdd = test.map{ case Tuple2(k, v) => Seq(k + integer) }.filter{ case Seq(v) => v < 30 } - RDDSchema(rdd.asInstanceOf[RDD[Seq[_]]], "myvalues int") - } -} diff --git a/src/main/test/scala/shark/SharkRunner.scala b/src/main/test/scala/shark/SharkRunner.scala deleted file mode 100755 index 01e69173..00000000 --- a/src/main/test/scala/shark/SharkRunner.scala +++ /dev/null @@ -1,127 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark - -import org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME - -import shark.api.JavaSharkContext -import shark.memstore2.MemoryMetadataManager - - -object SharkRunner { - - val WAREHOUSE_PATH = TestUtils.getWarehousePath() - val METASTORE_PATH = TestUtils.getMetastorePath() - val MASTER = "local" - - var sc: SharkContext = _ - - var javaSc: JavaSharkContext = _ - - def init(): SharkContext = synchronized { - if (sc == null) { - sc = SharkEnv.initWithSharkContext("shark-sql-suite-testing", MASTER) - - sc.runSql("set javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + - METASTORE_PATH + ";create=true") - sc.runSql("set hive.metastore.warehouse.dir=" + WAREHOUSE_PATH) - sc.runSql("set shark.test.data.path=" + TestUtils.dataFilePath) - - // second db - sc.sql("create database if not exists seconddb") - - loadTables() - } - sc - } - - def initWithJava(): JavaSharkContext = synchronized { - if (javaSc == null) { - javaSc = new JavaSharkContext(init()) - } - javaSc - } - - /** - * Tables accessible by any test. Their properties should remain constant across - * tests. - */ - def loadTables() = synchronized { - require(sc != null, "call init() to instantiate a SharkContext first") - - // Use the default namespace - sc.runSql("USE " + DEFAULT_DATABASE_NAME) - - // test - sc.runSql("drop table if exists test") - sc.runSql("CREATE TABLE test (key INT, val STRING)") - sc.runSql("LOAD DATA LOCAL INPATH '${hiveconf:shark.test.data.path}/kv1.txt' INTO TABLE test") - sc.runSql("drop table if exists test_cached") - sc.runSql("CREATE TABLE test_cached tblproperties('shark.cache'='memory') AS SELECT * FROM test") - - // test_null - sc.runSql("drop table if exists test_null") - sc.runSql("CREATE TABLE test_null (key INT, val STRING)") - sc.runSql("""LOAD DATA LOCAL INPATH '${hiveconf:shark.test.data.path}/kv3.txt' - INTO TABLE test_null""") - sc.runSql("drop table if exists test_null_cached") - sc.runSql("CREATE TABLE test_null_cached tblproperties('shark.cache'='memory') AS SELECT * FROM test_null") - - // clicks - sc.runSql("drop table if exists clicks") - sc.runSql("""create table clicks (id int, click int) - row format delimited fields terminated by '\t'""") - sc.runSql("""load data local inpath '${hiveconf:shark.test.data.path}/clicks.txt' - OVERWRITE INTO TABLE clicks""") - sc.runSql("drop table if exists clicks_cached") - sc.runSql("create table clicks_cached tblproperties('shark.cache'='memory') as select * from clicks") - - // users - sc.runSql("drop table if exists users") - sc.runSql("""create table users (id int, name string) - row format delimited fields terminated by '\t'""") - sc.runSql("""load data local inpath '${hiveconf:shark.test.data.path}/users.txt' - OVERWRITE INTO TABLE users""") - sc.runSql("drop table if exists users_cached") - sc.runSql("create table users_cached tblproperties('shark.cache'='memory') as select * from users") - - // test1 - sc.sql("drop table if exists test1") - sc.sql("""CREATE TABLE test1 (id INT, test1val ARRAY) - row format delimited fields terminated by '\t'""") - sc.sql("LOAD DATA LOCAL INPATH '${hiveconf:shark.test.data.path}/test1.txt' INTO TABLE test1") - sc.sql("drop table if exists test1_cached") - sc.sql("CREATE TABLE test1_cached tblproperties('shark.cache'='memory') AS SELECT * FROM test1") - Unit - } - - def expectSql(sql: String, expectedResults: Array[String], sort: Boolean = true) { - val sharkResults: Array[String] = sc.runSql(sql).results.map(_.mkString("\t")).toArray - val results = if (sort) sharkResults.sortWith(_ < _) else sharkResults - val expected = if (sort) expectedResults.sortWith(_ < _) else expectedResults - assert(results.corresponds(expected)(_.equals(_)), - "In SQL: " + sql + "\n" + - "Expected: " + expected.mkString("\n") + "; got " + results.mkString("\n")) - } - - // A shortcut for single row results. - def expectSql(sql: String, expectedResult: String) { - expectSql(sql, Array(expectedResult)) - } - -} diff --git a/src/main/test/scala/shark/SharkServerSuite.scala b/src/main/test/scala/shark/SharkServerSuite.scala deleted file mode 100755 index 686dc193..00000000 --- a/src/main/test/scala/shark/SharkServerSuite.scala +++ /dev/null @@ -1,108 +0,0 @@ -package shark - -import java.io.{BufferedReader, InputStreamReader} -import java.sql.DriverManager -import java.sql.Statement -import java.sql.Connection - -import scala.collection.JavaConversions._ - -import org.scalatest.{BeforeAndAfterAll, FunSuite} -import org.scalatest.matchers.ShouldMatchers - -import scala.concurrent._ -import ExecutionContext.Implicits.global - -/** - * Test for the Shark server. - */ -class SharkServerSuite extends FunSuite with BeforeAndAfterAll with ShouldMatchers with TestUtils { - - val WAREHOUSE_PATH = TestUtils.getWarehousePath("server") - val METASTORE_PATH = TestUtils.getMetastorePath("server") - val DRIVER_NAME = "org.apache.hadoop.hive.jdbc.HiveDriver" - val TABLE = "test" - // use a different port, than the hive standard 10000, - // for tests to avoid issues with the port being taken on some machines - val PORT = "9011" - - // If verbose is true, the testing program will print all outputs coming from the shark server. - val VERBOSE = Option(System.getenv("SHARK_TEST_VERBOSE")).getOrElse("false").toBoolean - - Class.forName(DRIVER_NAME) - - override def beforeAll() { launchServer() } - - override def afterAll() { stopServer() } - - private def launchServer(args: Seq[String] = Seq.empty) { - // Forking a new process to start the Shark server. The reason to do this is it is - // hard to clean up Hive resources entirely, so we just start a new process and kill - // that process for cleanup. - val defaultArgs = Seq("./bin/shark", "--service", "sharkserver", - "--verbose", - "-p", - PORT, - "--hiveconf", - "hive.root.logger=INFO,console", - "--hiveconf", - "\"javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=" + METASTORE_PATH + ";create=true\"", - "--hiveconf", - "\"hive.metastore.warehouse.dir=" + WAREHOUSE_PATH + "\"") - val pb = new ProcessBuilder(defaultArgs ++ args) - process = pb.start() - inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) - errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) - waitForOutput(inputReader, "Starting Shark server") - - // Spawn a thread to read the output from the forked process. - // Note that this is necessary since in some configurations, log4j could be blocked - // if its output to stderr are not read, and eventually blocking the entire test suite. - future { - while (true) { - val stdout = readFrom(inputReader) - val stderr = readFrom(errorReader) - if (VERBOSE && stdout.length > 0) { - println(stdout) - } - if (VERBOSE && stderr.length > 0) { - println(stderr) - } - Thread.sleep(50) - } - } - } - - private def stopServer() { - process.destroy() - process.waitFor() - } - - test("test query execution against a shark server") { - Thread.sleep(5*1000) // I know... Gross. However, without this the tests fail non-deterministically. - - val dataFilePath = TestUtils.dataFilePath + "/kv1.txt" - val stmt = createStatement() - stmt.executeQuery("DROP TABLE IF EXISTS test") - stmt.executeQuery("DROP TABLE IF EXISTS test_cached") - stmt.executeQuery("CREATE TABLE test(key int, val string)") - stmt.executeQuery("LOAD DATA LOCAL INPATH '" + dataFilePath+ "' OVERWRITE INTO TABLE test") - stmt.executeQuery("CREATE TABLE test_cached as select * from test limit 499") - - var rs = stmt.executeQuery("select count(*) from test") - rs.next() - rs.getInt(1) should equal (500) - - rs = stmt.executeQuery("select count(*) from test_cached") - rs.next() - rs.getInt(1) should equal (499) - - stmt.close() - } - - def getConnection(): Connection = { - DriverManager.getConnection("jdbc:hive://localhost:" + PORT + "/default", "", "") - } - - def createStatement(): Statement = getConnection().createStatement() -} \ No newline at end of file diff --git a/src/main/test/scala/shark/SortSuite.scala b/src/main/test/scala/shark/SortSuite.scala deleted file mode 100755 index ead1e4b5..00000000 --- a/src/main/test/scala/shark/SortSuite.scala +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark - -import org.apache.hadoop.io.BytesWritable - -import org.scalatest.FunSuite - -import org.apache.spark.SparkContext -import org.apache.spark.rdd.RDD - -import shark.execution.{ReduceKey, ReduceKeyMapSide, ReduceKeyReduceSide, RDDUtils} - - -class SortSuite extends FunSuite { - - TestUtils.init() - - var sc: SparkContext = SharkRunner.init() - - test("order by limit") { - val data = Array((4, 14), (1, 11), (7, 17), (0, 10)) - val expected = data.sortWith(_._1 < _._1).toSeq - val rdd: RDD[(ReduceKey, BytesWritable)] = sc.parallelize(data, 50).map { x => - (new ReduceKeyMapSide(new BytesWritable(Array[Byte](x._1.toByte))), - new BytesWritable(Array[Byte](x._2.toByte))) - } - for (k <- 0 to 5) { - val sortedRdd = RDDUtils.topK(rdd, k).asInstanceOf[RDD[(ReduceKeyReduceSide, Array[Byte])]] - val output = sortedRdd.map { case(k, v) => - (k.byteArray(0).toInt, v(0).toInt) - }.collect().toSeq - assert(output.size === math.min(k, 4)) - assert(output === expected.take(math.min(k, 4))) - } - } - -} diff --git a/src/main/test/scala/shark/TachyonSQLSuite.scala b/src/main/test/scala/shark/TachyonSQLSuite.scala deleted file mode 100755 index c808f242..00000000 --- a/src/main/test/scala/shark/TachyonSQLSuite.scala +++ /dev/null @@ -1,436 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark - -import java.util.{HashMap => JavaHashMap} - -import scala.collection.JavaConversions._ - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite - -import org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME -import org.apache.hadoop.hive.ql.metadata.Hive -import org.apache.spark.rdd.UnionRDD -import org.apache.spark.storage.StorageLevel - -import shark.api.QueryExecutionException -import shark.memstore2.{OffHeapStorageClient, CacheType, MemoryMetadataManager, PartitionedMemoryTable} -// import expectSql() shortcut methods -import shark.SharkRunner._ - - -class TachyonSQLSuite extends FunSuite with BeforeAndAfterAll { - - val DEFAULT_DB_NAME = DEFAULT_DATABASE_NAME - val KV1_TXT_PATH = "${hiveconf:shark.test.data.path}/kv1.txt" - - var sc: SharkContext = SharkRunner.init() - var sharkMetastore: MemoryMetadataManager = SharkEnv.memoryMetadataManager - - // Determine if Tachyon enabled at runtime. - val isTachyonEnabled = sys.env.contains("TACHYON_MASTER") - - override def beforeAll() { - if (isTachyonEnabled) { - sc.runSql("create table test_tachyon as select * from test") - } - } - - override def afterAll() { - if (isTachyonEnabled) { - sc.runSql("drop table test_tachyon") - } - } - - private def isTachyonTable( - dbName: String, - tableName: String, - hivePartitionKeyOpt: Option[String] = None): Boolean = { - val tableKey = MemoryMetadataManager.makeTableKey(dbName, tableName) - OffHeapStorageClient.client.tablePartitionExists(tableKey, hivePartitionKeyOpt) - } - - private def createPartitionedTachyonTable(tableName: String, numPartitionsToCreate: Int) { - sc.runSql("drop table if exists %s".format(tableName)) - sc.runSql(""" - create table %s(key int, value string) - partitioned by (keypart int) - tblproperties('shark.cache' = 'tachyon') - """.format(tableName)) - var partitionNum = 1 - while (partitionNum <= numPartitionsToCreate) { - sc.runSql("""insert into table %s partition(keypart = %d) - select * from test_tachyon""".format(tableName, partitionNum)) - partitionNum += 1 - } - assert(isTachyonTable(DEFAULT_DB_NAME, tableName)) - } - - if (isTachyonEnabled) { - ////////////////////////////////////////////////////////////////////////////// - // basic SQL - ////////////////////////////////////////////////////////////////////////////// - test("count") { - expectSql("select count(*) from test_tachyon", "500") - } - - test("filter") { - expectSql("select * from test_tachyon where key=100 or key=497", - Array("100\tval_100", "100\tval_100", "497\tval_497")) - } - - test("count distinct") { - sc.runSql("set mapred.reduce.tasks=3") - expectSql("select count(distinct key) from test_tachyon", "309") - expectSql( - """|SELECT substr(key,1,1), count(DISTINCT substr(val,5)) from test_tachyon - |GROUP BY substr(key,1,1)""".stripMargin, - Array("0\t1", "1\t71", "2\t69", "3\t62", "4\t74", "5\t6", "6\t5", "7\t6", "8\t8", "9\t7")) - } - - test("count bigint") { - sc.runSql("drop table if exists test_bigint") - sc.runSql("create table test_bigint (key bigint, val string)") - sc.runSql("""load data local inpath '${hiveconf:shark.test.data.path}/kv1.txt' - OVERWRITE INTO TABLE test_bigint""") - sc.runSql("drop table if exists test_bigint_tachyon") - sc.runSql("create table test_bigint_tachyon as select * from test_bigint") - expectSql("select val, count(*) from test_bigint_tachyon where key=484 group by val", - "val_484\t1") - - sc.runSql("drop table if exists test_bigint_tachyon") - } - - test("limit") { - assert(sc.runSql("select * from test_tachyon limit 10").results.length === 10) - assert(sc.runSql("select * from test_tachyon limit 501").results.length === 500) - sc.runSql("drop table if exists test_limit0_tachyon") - assert(sc.runSql("select * from test_tachyon limit 0").results.length === 0) - assert(sc.runSql("create table test_limit0_tachyon as select * from test_tachyon limit 0") - .results.length === 0) - assert(sc.runSql("select * from test_limit0_tachyon limit 0").results.length === 0) - assert(sc.runSql("select * from test_limit0_tachyon limit 1").results.length === 0) - - sc.runSql("drop table if exists test_limit0_tachyon") - } - - ////////////////////////////////////////////////////////////////////////////// - // cache DDL - ////////////////////////////////////////////////////////////////////////////// - test("Use regular CREATE TABLE and '_tachyon' suffix to create Tachyon table") { - sc.runSql("drop table if exists empty_table_tachyon") - sc.runSql("create table empty_table_tachyon(key string, value string)") - assert(isTachyonTable(DEFAULT_DB_NAME, "empty_table_tachyon")) - - sc.runSql("drop table if exists empty_table_tachyon") - } - - test("Use regular CREATE TABLE and table properties to create Tachyon table") { - sc.runSql("drop table if exists empty_table_tachyon_tbl_props") - sc.runSql("""create table empty_table_tachyon_tbl_props(key string, value string) - TBLPROPERTIES('shark.cache' = 'tachyon')""") - assert(isTachyonTable(DEFAULT_DB_NAME, "empty_table_tachyon_tbl_props")) - - sc.runSql("drop table if exists empty_table_tachyon_tbl_props") - } - - test("Insert into empty Tachyon table") { - sc.runSql("drop table if exists new_table_tachyon") - sc.runSql("create table new_table_tachyon(key string, value string)") - sc.runSql("insert into table new_table_tachyon select * from test where key > -1 limit 499") - expectSql("select count(*) from new_table_tachyon", "499") - - sc.runSql("drop table if exists new_table_tachyon") - } - - test("rename Tachyon table") { - sc.runSql("drop table if exists test_oldname_tachyon") - sc.runSql("drop table if exists test_rename") - sc.runSql("create table test_oldname_tachyon as select * from test") - sc.runSql("alter table test_oldname_tachyon rename to test_rename") - - assert(!isTachyonTable(DEFAULT_DB_NAME, "test_oldname_tachyon")) - assert(isTachyonTable(DEFAULT_DB_NAME, "test_rename")) - - expectSql("select count(*) from test_rename", "500") - - sc.runSql("drop table if exists test_rename") - } - - test("insert into tachyon tables") { - sc.runSql("drop table if exists test1_tachyon") - sc.runSql("create table test1_tachyon as select * from test") - expectSql("select count(*) from test1_tachyon", "500") - sc.runSql("insert into table test1_tachyon select * from test where key > -1 limit 499") - expectSql("select count(*) from test1_tachyon", "999") - - sc.runSql("drop table if exists test1_tachyon") - } - - test("insert overwrite") { - sc.runSql("drop table if exists test2_tachyon") - sc.runSql("create table test2_tachyon as select * from test") - expectSql("select count(*) from test2_tachyon", "500") - sc.runSql("insert overwrite table test2_tachyon select * from test where key > -1 limit 499") - expectSql("select count(*) from test2_tachyon", "499") - - sc.runSql("drop table if exists test2_tachyon") - } - - test("error when attempting to update Tachyon table(s) using command with multiple INSERTs") { - sc.runSql("drop table if exists multi_insert_test") - sc.runSql("drop table if exists multi_insert_test_tachyon") - sc.runSql("create table multi_insert_test as select * from test") - sc.runSql("create table multi_insert_test_tachyon as select * from test") - intercept[QueryExecutionException] { - sc.runSql("""from test - insert into table multi_insert_test select * - insert into table multi_insert_test_tachyon select *""") - } - - sc.runSql("drop table if exists multi_insert_test") - sc.runSql("drop table if exists multi_insert_test_tachyon") - } - - test("create Tachyon table with 'shark.cache' flag in table properties") { - sc.runSql("drop table if exists ctas_tbl_props") - sc.runSql("""create table ctas_tbl_props TBLPROPERTIES ('shark.cache'='tachyon') as - select * from test""") - assert(isTachyonTable(DEFAULT_DB_NAME, "ctas_tbl_props")) - expectSql("select * from ctas_tbl_props where key=407", "407\tval_407") - - sc.runSql("drop table if exists ctas_tbl_props") - } - - test("tachyon tables with complex types") { - sc.runSql("drop table if exists test_complex_types") - sc.runSql("drop table if exists test_complex_types_tachyon") - sc.runSql("""CREATE TABLE test_complex_types ( - a STRING, b ARRAY, c ARRAY>, d MAP>)""") - sc.runSql("""load data local inpath '${hiveconf:shark.test.data.path}/create_nested_type.txt' - overwrite into table test_complex_types""") - sc.runSql("""create table test_complex_types_tachyon TBLPROPERTIES ("shark.cache" = "tachyon") - as select * from test_complex_types""") - - assert(sc.sql("select a from test_complex_types_tachyon where a = 'a0'").head === "a0") - - assert(sc.sql("select b from test_complex_types_tachyon where a = 'a0'").head === - """["b00","b01"]""") - - assert(sc.sql("select c from test_complex_types_tachyon where a = 'a0'").head === - """[{"c001":"C001","c002":"C002"},{"c011":null,"c012":"C012"}]""") - - assert(sc.sql("select d from test_complex_types_tachyon where a = 'a0'").head === - """{"d01":["d011","d012"],"d02":["d021","d022"]}""") - - assert(isTachyonTable(DEFAULT_DB_NAME, "test_complex_types_tachyon")) - - sc.runSql("drop table if exists test_complex_types") - sc.runSql("drop table if exists test_complex_types_tachyon") - } - - test("disable caching in Tachyon by default") { - sc.runSql("set shark.cache.flag.checkTableName=false") - sc.runSql("drop table if exists should_not_be_in_tachyon") - sc.runSql("create table should_not_be_in_tachyon as select * from test") - expectSql("select key from should_not_be_in_tachyon where key = 407", "407") - assert(!isTachyonTable(DEFAULT_DB_NAME, "should_not_be_in_tachyon")) - - sc.runSql("set shark.cache.flag.checkTableName=true") - sc.runSql("drop table if exists should_not_be_in_tachyon") - } - - test("tachyon table name should be case-insensitive") { - sc.runSql("drop table if exists sharkTest5tachyon") - sc.runSql("""create table sharkTest5tachyon TBLPROPERTIES ("shark.cache" = "tachyon") as - select * from test""") - expectSql("select val from sharktest5tachyon where key = 407", "val_407") - assert(isTachyonTable(DEFAULT_DB_NAME, "sharkTest5tachyon")) - - sc.runSql("drop table if exists sharkTest5tachyon") - } - - test("dropping tachyon tables should clean up RDDs") { - sc.runSql("drop table if exists sharkTest5tachyon") - sc.runSql("""create table sharkTest5tachyon TBLPROPERTIES ("shark.cache" = "tachyon") as - select * from test""") - sc.runSql("drop table sharkTest5tachyon") - assert(!isTachyonTable(DEFAULT_DB_NAME, "sharkTest5tachyon")) - } - - ////////////////////////////////////////////////////////////////////////////// - // Caching Hive-partititioned tables - // Note: references to 'partition' for this section refer to a Hive-partition. - ////////////////////////////////////////////////////////////////////////////// - test("Use regular CREATE TABLE and '_tachyon' suffix to create partitioned Tachyon table") { - sc.runSql("drop table if exists empty_part_table_tachyon") - sc.runSql("""create table empty_part_table_tachyon(key int, value string) - partitioned by (keypart int)""") - assert(isTachyonTable(DEFAULT_DB_NAME, "empty_part_table_tachyon")) - - sc.runSql("drop table if exists empty_part_table_tachyon") - } - - test("Use regular CREATE TABLE and table properties to create partitioned Tachyon table") { - sc.runSql("drop table if exists empty_part_table_tachyon_tbl_props") - sc.runSql("""create table empty_part_table_tachyon_tbl_props(key int, value string) - partitioned by (keypart int) tblproperties('shark.cache' = 'tachyon')""") - assert(isTachyonTable(DEFAULT_DB_NAME, "empty_part_table_tachyon_tbl_props")) - - sc.runSql("drop table if exists empty_part_table_tachyon_tbl_props") - } - - test("alter Tachyon table by adding a new partition") { - sc.runSql("drop table if exists alter_part_tachyon") - sc.runSql("""create table alter_part_tachyon(key int, value string) - partitioned by (keypart int)""") - sc.runSql("""alter table alter_part_tachyon add partition(keypart = 1)""") - val tableName = "alter_part_tachyon" - val partitionColumn = "keypart=1" - assert(isTachyonTable(DEFAULT_DB_NAME, "alter_part_tachyon", Some(partitionColumn))) - - sc.runSql("drop table if exists alter_part_tachyon") - } - - test("alter Tachyon table by dropping a partition") { - sc.runSql("drop table if exists alter_drop_tachyon") - sc.runSql("""create table alter_drop_tachyon(key int, value string) - partitioned by (keypart int)""") - sc.runSql("""alter table alter_drop_tachyon add partition(keypart = 1)""") - - val tableName = "alter_drop_tachyon" - val partitionColumn = "keypart=1" - assert(isTachyonTable(DEFAULT_DB_NAME, "alter_drop_tachyon", Some(partitionColumn))) - sc.runSql("""alter table alter_drop_tachyon drop partition(keypart = 1)""") - assert(!isTachyonTable(DEFAULT_DB_NAME, "alter_drop_tachyon", Some(partitionColumn))) - - sc.runSql("drop table if exists alter_drop_tachyon") - } - - test("insert into a partition of a Tachyon table") { - val tableName = "insert_part_tachyon" - createPartitionedTachyonTable( - tableName, - numPartitionsToCreate = 1) - expectSql("select value from insert_part_tachyon where key = 407 and keypart = 1", "val_407") - - sc.runSql("drop table if exists insert_part_tachyon") - } - - test("insert overwrite a partition of a Tachyon table") { - val tableName = "insert_over_part_tachyon" - createPartitionedTachyonTable( - tableName, - numPartitionsToCreate = 1) - expectSql("""select value from insert_over_part_tachyon - where key = 407 and keypart = 1""", "val_407") - sc.runSql("""insert overwrite table insert_over_part_tachyon partition(keypart = 1) - select key, -1 from test""") - expectSql("select value from insert_over_part_tachyon where key = 407 and keypart = 1", "-1") - - sc.runSql("drop table if exists insert_over_part_tachyon") - } - - test("scan partitioned Tachyon table that's empty") { - sc.runSql("drop table if exists empty_part_table_tachyon") - sc.runSql("""create table empty_part_table_tachyon(key int, value string) - partitioned by (keypart int)""") - expectSql("select count(*) from empty_part_table_tachyon", "0") - - sc.runSql("drop table if exists empty_part_table_tachyon") - } - - test("scan partitioned Tachyon table that has a single partition") { - val tableName = "scan_single_part_tachyon" - createPartitionedTachyonTable( - tableName, - numPartitionsToCreate = 1) - expectSql("select * from scan_single_part_tachyon where key = 407", "407\tval_407\t1") - - sc.runSql("drop table if exists scan_single_part_tachyon") - } - - test("scan partitioned Tachyon table that has multiple partitions") { - val tableName = "scan_mult_part_tachyon" - createPartitionedTachyonTable( - tableName, - numPartitionsToCreate = 3) - expectSql("select * from scan_mult_part_tachyon where key = 407 order by keypart", - Array("407\tval_407\t1", "407\tval_407\t2", "407\tval_407\t3")) - - sc.runSql("drop table if exists scan_mult_part_tachyon") - } - - test("drop/unpersist partitioned Tachyon table that has multiple partitions") { - val tableName = "drop_mult_part_tachyon" - createPartitionedTachyonTable( - tableName, - numPartitionsToCreate = 3) - expectSql("select count(1) from drop_mult_part_tachyon", "1500") - sc.runSql("drop table drop_mult_part_tachyon ") - assert(!isTachyonTable(DEFAULT_DB_NAME, tableName)) - - sc.runSql("drop table if exists drop_mult_part_tachyon") - } - - ///////////////////////////////////////////////////////////////////////////// - // LOAD for Tachyon tables - ////////////////////////////////////////////////////////////////////////////// - test ("LOAD INTO a Tachyon table") { - sc.runSql("drop table if exists load_into_tachyon") - sc.runSql("create table load_into_tachyon (key int, value string)") - sc.runSql("load data local inpath '%s' into table load_into_tachyon".format(KV1_TXT_PATH)) - expectSql("select count(*) from load_into_tachyon", "500") - - sc.runSql("drop table if exists load_into_tachyon") - } - - test ("LOAD OVERWRITE a Tachyon table") { - sc.runSql("drop table if exists load_overwrite_tachyon") - sc.runSql("create table load_overwrite_tachyon (key int, value string)") - sc.runSql("load data local inpath '%s' into table load_overwrite_tachyon". - format("${hiveconf:shark.test.data.path}/kv3.txt")) - expectSql("select count(*) from load_overwrite_tachyon", "25") - sc.runSql("load data local inpath '%s' overwrite into table load_overwrite_tachyon". - format(KV1_TXT_PATH)) - expectSql("select count(*) from load_overwrite_tachyon", "500") - sc.runSql("drop table if exists load_overwrite_tachyon") - } - - test ("LOAD INTO a partitioned Tachyon table") { - sc.runSql("drop table if exists load_into_part_tachyon") - sc.runSql("""create table load_into_part_tachyon (key int, value string) - partitioned by (keypart int)""") - sc.runSql("""load data local inpath '%s' into table load_into_part_tachyon - partition(keypart = 1)""".format(KV1_TXT_PATH)) - expectSql("select count(*) from load_into_part_tachyon", "500") - sc.runSql("drop table if exists load_into_part_tachyon") - } - - test ("LOAD OVERWRITE a partitioned Tachyon table") { - sc.runSql("drop table if exists load_overwrite_part_tachyon") - sc.runSql("""create table load_overwrite_part_tachyon (key int, value string) - partitioned by (keypart int)""") - sc.runSql("""load data local inpath '%s' overwrite into table load_overwrite_part_tachyon - partition(keypart = 1)""".format(KV1_TXT_PATH)) - expectSql("select count(*) from load_overwrite_part_tachyon", "500") - sc.runSql("drop table if exists load_overwrite_part_tachyon") - } - } -} diff --git a/src/main/test/scala/shark/TestUtils.scala b/src/main/test/scala/shark/TestUtils.scala deleted file mode 100755 index 25d9b4af..00000000 --- a/src/main/test/scala/shark/TestUtils.scala +++ /dev/null @@ -1,195 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark - -import java.io.{BufferedReader, InputStreamReader, PrintWriter} -import java.text.SimpleDateFormat -import java.util.{Date, HashMap => JHashMap} - -import org.apache.hadoop.hive.common.LogUtils -import org.apache.hadoop.hive.common.LogUtils.LogInitializationException - -import org.apache.spark.rdd.{RDD, UnionRDD} -import org.apache.spark.storage.StorageLevel - - -object TestUtils { - - val timestamp = new SimpleDateFormat("yyyyMMdd-HHmmss") - - def getWarehousePath(prefix: String): String = { - System.getProperty("user.dir") + "/test_warehouses/" + prefix + "-warehouse-" + - timestamp.format(new Date) - } - - def getMetastorePath(prefix: String): String = { - System.getProperty("user.dir") + "/test_warehouses/" + prefix + "-metastore-" + - timestamp.format(new Date) - } - - def testAndSet(prefix: String): Boolean = synchronized { - if (testAndTestMap.get(prefix) == null) { - testAndTestMap.put(prefix, new Object) - true - } else { - false - } - } - - def getStorageLevelOfRDD(rdd: RDD[_]): StorageLevel = { - rdd match { - case u: UnionRDD[_] => { - // Find the storage level of a UnionRDD from the storage levels of RDDs that compose it. - // A StorageLevel.NONE is returned if all of those RDDs have StorageLevel.NONE. - // Mutually recursive if any RDD in 'u.rdds' is a UnionRDD. - getStorageLevelOfRDDs(u.rdds) - } - case _ => rdd.getStorageLevel - } - } - - /** - * Returns the storage level of a sequence of RDDs, interpreted as the storage level of the first - * RDD in the sequence that is persisted in memory or on disk. This works because for Shark's use - * case, all RDDs for a non-partitioned table should have the same storage level. An RDD for a - * partitioned table could be StorageLevel.NONE if it was unpersisted by the partition eviction - * policy. - * - * @param rdds The sequence of RDDs to find the StorageLevel of. - */ - def getStorageLevelOfRDDs(rdds: Seq[RDD[_]]): StorageLevel = { - rdds.foldLeft(StorageLevel.NONE) { - (s, r) => { - if (s == StorageLevel.NONE) { - // Mutally recursive if `r` is a UnionRDD. However, this shouldn't happen in Shark, since - // UnionRDDs from successive INSERT INTOs are created through #unionAndFlatten(). - getStorageLevelOfRDD(r) - } else { - // Some RDD in 'rdds' is persisted in memory or disk, so return early. - return s - } - } - } - } - - // Don't use default arguments in the above functions because otherwise the JavaAPISuite - // can't call those functions with default arguments. - def getWarehousePath(): String = getWarehousePath("sql") - def getMetastorePath(): String = getMetastorePath("sql") - def testAndSet(): Boolean = testAndSet("sql") - - private val testAndTestMap = new JHashMap[String, Object] - - def dataFilePath: String = { - Option(System.getenv("SHARK_HOME")).getOrElse(System.getProperty("user.dir")) + "/data/files" - } - - // Dummy function for initialize the log4j properties. - def init() { } - - // initialize log4j - try { - LogUtils.initHiveLog4j() - } catch { - case e: LogInitializationException => // Ignore the error. - } -} - - -trait TestUtils { - - var process : Process = null - var outputWriter : PrintWriter = null - var inputReader : BufferedReader = null - var errorReader : BufferedReader = null - - def dropTable(tableName: String, timeout: Long = 15000): String = { - executeQuery("drop table if exists " + tableName + ";") - } - - def executeQuery( - cmd: String, outputMessage: String = "OK", timeout: Long = 15000): String = { - println("Executing: " + cmd + ", expecting output: " + outputMessage) - outputWriter.write(cmd + "\n") - outputWriter.flush() - waitForQuery(timeout, outputMessage) - } - - protected def waitForQuery(timeout: Long, message: String): String = { - if (waitForOutput(errorReader, message, timeout)) { - Thread.sleep(500) - readOutput() - } else { - assert(false, "Didn't find \"" + message + "\" in the output:\n" + readOutput()) - null - } - } - - protected def waitForQuery(timeout: Long, message1: String, message2: String): String = { - if (waitForOutput2(errorReader, message1, message2, timeout)) { - Thread.sleep(500) - readOutput() - } else { - assert(false, "Didn't find '" + message1 + "' or '" + message2 + - "' in the output:\n" + readOutput()) - null - } - } - - // Wait for the specified str to appear in the output. - protected def waitForOutput( - reader: BufferedReader, str: String, timeout: Long = 10000): Boolean = { - val startTime = System.currentTimeMillis - var out = "" - while (!out.contains(str) && (System.currentTimeMillis) < (startTime + timeout)) { - out += readFrom(reader) - } - out.contains(str) - } - - // Wait for the specified str1 and str2 to appear in the output. - protected def waitForOutput2( - reader: BufferedReader, str1: String, str2: String, timeout: Long = 10000): Boolean = { - val startTime = System.currentTimeMillis - var out = "" - while (!out.contains(str1) && !out.contains(str2) && - (System.currentTimeMillis) < (startTime + timeout)) { - out += readFrom(reader) - } - out.contains(str1) || out.contains(str2) - } - - // Read stdout output from Shark and filter out garbage collection messages. - protected def readOutput(): String = { - val output = readFrom(inputReader) - // Remove GC Messages - val filteredOutput = output.lines.filterNot(x => x.contains("[GC") || x.contains("[Full GC")) - .mkString("\n") - filteredOutput - } - - protected def readFrom(reader: BufferedReader): String = { - var out = "" - var c = 0 - while (reader.ready) { - c = reader.read() - out += c.asInstanceOf[Char] - } - out - } -} diff --git a/src/main/test/scala/shark/UtilsSuite.scala b/src/main/test/scala/shark/UtilsSuite.scala deleted file mode 100755 index 7cb48a18..00000000 --- a/src/main/test/scala/shark/UtilsSuite.scala +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark - -import java.util.{HashMap => JHashMap} - -import org.apache.hadoop.conf.Configuration - -import org.scalatest.{BeforeAndAfter, FunSuite} - - -class UtilsSuite extends FunSuite { - - test("set aws credentials") { - var conf = new Configuration - var map = new JHashMap[String, String]() - Utils.setAwsCredentials(conf, map) - assert(conf.get("fs.s3n.awsAccessKeyId") === null) - assert(conf.get("fs.s3n.awsSecretAccessKey") === null) - assert(conf.get("fs.s3.awsAccessKeyId") === null) - assert(conf.get("fs.s3.awsSecretAccessKey") === null) - - map.put("AWS_ACCESS_KEY_ID", "id") - conf = new Configuration - Utils.setAwsCredentials(conf, map) - assert(conf.get("fs.s3n.awsAccessKeyId") === null) - assert(conf.get("fs.s3n.awsSecretAccessKey") === null) - assert(conf.get("fs.s3.awsAccessKeyId") === null) - assert(conf.get("fs.s3.awsSecretAccessKey") === null) - - map.put("AWS_SECRET_ACCESS_KEY", "key") - conf = new Configuration - Utils.setAwsCredentials(conf, map) - assert(conf.get("fs.s3n.awsAccessKeyId") === "id") - assert(conf.get("fs.s3n.awsSecretAccessKey") === "key") - assert(conf.get("fs.s3.awsAccessKeyId") === "id") - assert(conf.get("fs.s3.awsSecretAccessKey") === "key") - } - -} diff --git a/src/main/test/scala/shark/execution/HiveStructSerializerSuite.scala b/src/main/test/scala/shark/execution/HiveStructSerializerSuite.scala deleted file mode 100755 index ac646954..00000000 --- a/src/main/test/scala/shark/execution/HiveStructSerializerSuite.scala +++ /dev/null @@ -1,65 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark.execution - -import java.util.{ArrayList => JArrayList} - -import scala.collection.JavaConversions._ - -import org.apache.hadoop.hive.serde2.binarysortable.{HiveStructSerializer, HiveStructDeserializer} -import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, - ObjectInspectorFactory, StandardListObjectInspector, StandardMapObjectInspector, - StructObjectInspector} -import org.apache.hadoop.hive.serde2.objectinspector.primitive.{PrimitiveObjectInspectorUtils, - PrimitiveObjectInspectorFactory} -import org.apache.hadoop.io.{IntWritable, LongWritable, Text} - -import org.scalatest.FunSuite - - -class HiveStructSerializerSuite extends FunSuite { - - test("Testing serializing a simple row") { - val row1 = createRow(1, "test1") - val row2 = createRow(2, "test2") - val ser = new HiveStructSerializer(createObjectInspector) - val deser = new HiveStructDeserializer(createObjectInspector) - val deserRow1 = deser.deserialize(ser.serialize(row1)) - assert(row1.get(0).equals(deserRow1.get(0))) - assert(row1.get(1).equals(deserRow1.get(1))) - } - - def createObjectInspector(): StructObjectInspector = { - val names = List("a", "b") - val ois = List( - createPrimitiveOi(classOf[java.lang.Integer]), - createPrimitiveOi(classOf[String])) - ObjectInspectorFactory.getStandardStructObjectInspector(names, ois) - } - - def createRow(v1: Int, v2: String): JArrayList[Object] = { - val row = new JArrayList[Object](2) - row.add(new IntWritable(v1)) - row.add(new Text(v2)) - row - } - - def createPrimitiveOi(javaClass: Class[_]): PrimitiveObjectInspector = - PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector( - PrimitiveObjectInspectorUtils.getTypeEntryFromPrimitiveJavaClass(javaClass).primitiveCategory) -} diff --git a/src/main/test/scala/shark/execution/serialization/SerializationSuite.scala b/src/main/test/scala/shark/execution/serialization/SerializationSuite.scala deleted file mode 100755 index 297b7379..00000000 --- a/src/main/test/scala/shark/execution/serialization/SerializationSuite.scala +++ /dev/null @@ -1,109 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark.execution.serialization - -import scala.collection.JavaConversions._ -import scala.collection.mutable.ArrayBuffer - -import org.apache.commons.lang.ObjectUtils -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector -import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory -import org.apache.hadoop.hive.conf.HiveConf - -import org.scalatest.FunSuite - -import org.apache.spark.SparkConf -import org.apache.spark.serializer.{JavaSerializer => SparkJavaSerializer} - - -object SerializationSuite { - val DEPRECATED_HIVECONF_PROPERTIES = List( - "topology.script.file.name", - "topology.script.number.args", - "hadoop.configured.node.mapping", - "topology.node.switch.mapping.impl", - "dfs.df.interval", - "dfs.client.buffer.dir", - "hadoop.native.lib", - "fs.default.name" - ).toSet[String] -} - -class SerializationSuite extends FunSuite { - - test("Java serializing object inspectors") { - - val oi = PrimitiveObjectInspectorFactory.javaStringObjectInspector - val ois = KryoSerializationWrapper(new ArrayBuffer[ObjectInspector]) - ois.value += oi - - val ser = new SparkJavaSerializer(new SparkConf(loadDefaults = false)) - val bytes = ser.newInstance().serialize(ois) - val desered = ser.newInstance() - .deserialize[KryoSerializationWrapper[ArrayBuffer[ObjectInspector]]](bytes) - - assert(desered.head.getTypeName() === oi.getTypeName()) - } - - test("Java serializing operators") { - import shark.execution.{FileSinkOperator => SharkFileSinkOperator} - - val operator = new SharkFileSinkOperator - operator.localHconf = new org.apache.hadoop.hive.conf.HiveConf - operator.localHiveOp = new org.apache.hadoop.hive.ql.exec.FileSinkOperator - val opWrapped = OperatorSerializationWrapper(operator) - - val ser = new SparkJavaSerializer(new SparkConf(loadDefaults = false)) - val bytes = ser.newInstance().serialize(opWrapped) - val desered = ser.newInstance() - .deserialize[OperatorSerializationWrapper[SharkFileSinkOperator]](bytes) - - assert(desered.value != null) - assert(desered.value.localHconf != null) - assert(desered.value.localHiveOp != null) - assertHiveConfEquals(opWrapped.localHconf, desered.value.localHconf) - } - - test("XMLEncoder HiveConfPersistenceDelegate") { - val hiveConf = new HiveConf - hiveConf.setAuxJars("hive-aux.jar") - hiveConf.set("test.test.test", "true") - val bytes = XmlSerializer.serialize(hiveConf, hiveConf) - val deseredConf = XmlSerializer.deserialize[org.apache.hadoop.hive.conf.HiveConf](bytes) - - assertHiveConfEquals(hiveConf, deseredConf) - } - - def assertHiveConfEquals(expectedConf: HiveConf, resultConf: HiveConf) = { - expectedConf.getAllProperties.entrySet().foreach { entry => - if (!SerializationSuite.DEPRECATED_HIVECONF_PROPERTIES.contains(entry.getKey)) { - assert(resultConf.getAllProperties.get(entry.getKey()) === entry.getValue) - } - } - - resultConf.getAllProperties.entrySet().foreach { entry => - if (!SerializationSuite.DEPRECATED_HIVECONF_PROPERTIES.contains(entry.getKey)) { - assert(expectedConf.getAllProperties.get(entry.getKey()) === entry.getValue) - } - } - assert(ObjectUtils.equals(expectedConf.getAuxJars, resultConf.getAuxJars)) - } - -} - - diff --git a/src/main/test/scala/shark/execution/serialization/ShuffleSerializerSuite.scala b/src/main/test/scala/shark/execution/serialization/ShuffleSerializerSuite.scala deleted file mode 100755 index aafa3d12..00000000 --- a/src/main/test/scala/shark/execution/serialization/ShuffleSerializerSuite.scala +++ /dev/null @@ -1,112 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark.execution.serialization - -import java.io.{ByteArrayInputStream, ByteArrayOutputStream} - -import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} - -import org.apache.hadoop.io.BytesWritable -import org.scalatest.FunSuite -import org.scalatest.matchers.ShouldMatchers - -import shark.execution.{ReduceKey, ReduceKeyMapSide, ReduceKeyReduceSide} - - -class ShuffleSerializerSuite extends FunSuite with ShouldMatchers { - test("Encoding and decoding variable ints") { - val check = List[Int](0, 1000, 10000, 100000, 1000000, 10000000, 100000000, 1000000000) - - val bos = new ByteArrayOutputStream() - val ser = new ShuffleSerializer() - val serOutStream = { - ser.newInstance().serializeStream(bos).asInstanceOf[ShuffleSerializationStream] - } - for (i <- check) { - serOutStream.writeUnsignedVarInt(i) - } - serOutStream.close(); - - val bis = new ByteArrayInputStream(bos.toByteArray) - val serInStream = { - ser.newInstance().deserializeStream(bis).asInstanceOf[ShuffleDeserializationStream] - } - for (in <- check) { - val out: Int = serInStream.readUnsignedVarInt() - assert(out == in, "Encoded: " + in + " did not match decoded: " + out) - } - } - - test("Serializing and deserializing from a stream") { - val NUM_ITEMS = 5000 - val KEY_SIZE = 1000 - val VALUE_SIZE = 1000 - - val initialItems: Array[(ReduceKey, BytesWritable)] = Array.fill(NUM_ITEMS) { - val rkBytes = (1 to KEY_SIZE).map(_.toByte).toArray - val valueBytes = (1 to VALUE_SIZE).map(_.toByte).toArray - val rk = new ReduceKeyMapSide(new BytesWritable(rkBytes)) - val value = new BytesWritable(valueBytes) - (rk, value) - } - - val bos = new ByteArrayOutputStream() - val ser = new ShuffleSerializer() - val serStream = ser.newInstance().serializeStream(bos) - initialItems.map(serStream.writeObject(_)) - val bis = new ByteArrayInputStream(bos.toByteArray) - val serInStream = ser.newInstance().deserializeStream(bis) - - initialItems.foreach { expected: (ReduceKey, BytesWritable) => - val output: (ReduceKey, Array[Byte]) = serInStream.readObject() - (expected._1) should equal (output._1) - (expected._2.getBytes) should equal (output._2) - } - } - - test("Serializing and deserializing from a stream (with compression)") { - val NUM_ITEMS = 1000 - val KEY_SIZE = 1000 - val VALUE_SIZE = 1000 - - val initialItems = Array.fill(NUM_ITEMS) { - val rkBytes = (1 to KEY_SIZE).map(_.toByte).toArray - val valueBytes = (1 to VALUE_SIZE).map(_.toByte).toArray - val rk = new ReduceKeyMapSide(new BytesWritable(rkBytes)) - val value = new BytesWritable(valueBytes) - (rk, value) - } - - val bos = new ByteArrayOutputStream() - val cBos = new LZFOutputStream(bos).setFinishBlockOnFlush(true) - val ser = new ShuffleSerializer() - val serStream = ser.newInstance().serializeStream(cBos) - initialItems.map(serStream.writeObject(_)) - serStream.close() - val array = bos.toByteArray - val bis = new ByteArrayInputStream(array) - val cBis = new LZFInputStream(bis) - val serInStream = ser.newInstance().deserializeStream(cBis) - - initialItems.foreach { expected: (ReduceKey, BytesWritable) => - val output: (ReduceKey, Array[Byte]) = serInStream.readObject() - (expected._1) should equal (output._1) - (expected._2.getBytes) should equal (output._2) - } - } -} diff --git a/src/main/test/scala/shark/memstore2/CachePolicySuite.scala b/src/main/test/scala/shark/memstore2/CachePolicySuite.scala deleted file mode 100755 index 6eed3518..00000000 --- a/src/main/test/scala/shark/memstore2/CachePolicySuite.scala +++ /dev/null @@ -1,131 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark.memstore2 - -import org.scalatest.FunSuite - -import scala.collection.mutable.HashMap - -class CachePolicySuite extends FunSuite { - - case class TestValue(var value: Int, var isCached: Boolean) - - class IdentifyKVGen(max: Int) { - val kvMap = new HashMap[Int, TestValue]() - for (i <- 0 until max) { - kvMap(i) = TestValue(i, isCached = false) - } - - def loadFunc(key: Int) = { - val value = kvMap(key) - value.isCached = true - value - } - - def evictionFunc(key: Int, value: TestValue) = { - value.isCached = false - } - } - - test("LRU policy") { - val kvGen = new IdentifyKVGen(20) - val cacheSize = 10 - val lru = new LRUCachePolicy[Int, TestValue]() - lru.initialize(Array.empty[String], cacheSize, kvGen.loadFunc _, kvGen.evictionFunc _) - - // Load KVs 0-9. - (0 to 9).map(lru.notifyGet(_)) - assert(lru.keysOfCachedEntries.equals(Seq(0, 1, 2, 3, 4, 5, 6, 7, 8, 9))) - - // Reorder access order by getting keys 2-4. - (2 to 4).map(lru.notifyGet(_)) - assert(lru.keysOfCachedEntries.equals(Seq(0, 1, 5, 6, 7, 8, 9, 2, 3, 4))) - - // Get keys 10-12, which should evict (0, 1, 5). - (10 to 12).map(lru.notifyGet(_)) - assert(lru.keysOfCachedEntries.equals(Seq(6, 7, 8, 9, 2, 3, 4, 10, 11, 12))) - // Make sure the eviction function ran. - assert(!kvGen.kvMap(0).isCached) - assert(!kvGen.kvMap(1).isCached) - assert(!kvGen.kvMap(5).isCached) - - // Reorder access order by getting keys (6, 8, 2). - lru.notifyGet(6); lru.notifyGet(8); lru.notifyGet(2) - assert(lru.keysOfCachedEntries.equals(Seq(7, 9, 3, 4, 10, 11, 12, 6, 8, 2))) - - // Remove 9, 4 and add 13, 14, 15. 7 should be evicted. - lru.notifyRemove(9); lru.notifyRemove(4) - (13 to 15).map(lru.notifyGet(_)) - assert(lru.keysOfCachedEntries.equals(Seq(3, 10, 11, 12, 6, 8, 2, 13, 14, 15))) - assert(!kvGen.kvMap(7).isCached) - } - - test("FIFO policy") { - val kvGen = new IdentifyKVGen(15) - val cacheSize = 5 - val fifo = new FIFOCachePolicy[Int, TestValue]() - fifo.initialize(Array.empty[String], cacheSize, kvGen.loadFunc _, kvGen.evictionFunc _) - - // Load KVs 0-4. - (0 to 4).map(fifo.notifyPut(_, TestValue(-1, true))) - assert(fifo.keysOfCachedEntries.equals(Seq(0, 1, 2, 3, 4))) - - // Get 0-8, which should evict 0-3. - (0 to 8).map(fifo.notifyPut(_, TestValue(-1, true))) - assert(fifo.keysOfCachedEntries.equals(Seq(4, 5, 6, 7, 8))) - - // Remove 4, 6 and add 9-12. 5 and 7 should be evicted. - fifo.notifyRemove(4); fifo.notifyRemove(6) - (9 to 12).map(fifo.notifyPut(_, TestValue(-1, true))) - assert(fifo.keysOfCachedEntries.equals(Seq(8, 9, 10, 11, 12))) - } - - test("Policy classes instantiated from a string, with maxSize argument") { - val kvGen = new IdentifyKVGen(15) - val lruStr = "shark.memstore2.LRUCachePolicy(5)" - val lru = CachePolicy.instantiateWithUserSpecs( - lruStr, fallbackMaxSize = 10, kvGen.loadFunc _, kvGen.evictionFunc _) - assert(lru.maxSize == 5) - val fifoStr = "shark.memstore2.FIFOCachePolicy(5)" - val fifo = CachePolicy.instantiateWithUserSpecs( - fifoStr, fallbackMaxSize = 10, kvGen.loadFunc _, kvGen.evictionFunc _) - assert(fifo.maxSize == 5) - } - - test("Cache stats are recorded") { - val kvGen = new IdentifyKVGen(20) - val cacheSize = 5 - val lru = new LRUCachePolicy[Int, TestValue]() - lru.initialize(Array.empty[String], cacheSize, kvGen.loadFunc _, kvGen.evictionFunc _) - - // Hit rate should start at 1.0 - assert(lru.hitRate == 1.0) - - (0 to 4).map(lru.notifyGet(_)) - assert(lru.hitRate == 0.0) - - // Get 1, 2, 3, which should bring the hit rate to 0.375. - (1 to 3).map(lru.notifyGet(_)) - assert(lru.hitRate == 0.375) - - // Get 2-5, which brings the hit rate up to 0.50. - (2 to 5).map(lru.notifyGet(_)) - assert(lru.evictionCount == 1) - assert(lru.hitRate == 0.50) - } -} \ No newline at end of file diff --git a/src/main/test/scala/shark/memstore2/ColumnIteratorSuite.scala b/src/main/test/scala/shark/memstore2/ColumnIteratorSuite.scala deleted file mode 100755 index 8c297cac..00000000 --- a/src/main/test/scala/shark/memstore2/ColumnIteratorSuite.scala +++ /dev/null @@ -1,413 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark.memstore2 - -import org.apache.hadoop.hive.serde2.`lazy`.ByteArrayRef -import org.apache.hadoop.hive.serde2.objectinspector.primitive._ -import org.apache.hadoop.io._ -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector - -import org.scalatest.FunSuite - -import shark.memstore2.column._ -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector -import shark.memstore2.column.Implicits._ -import java.nio.ByteOrder - - -class ColumnIteratorSuite extends FunSuite { - - val PARALLEL_MODE = true - - test("void column") { - val builder = new VoidColumnBuilder - builder.initialize(5, "void") - builder.append(null, null) - builder.append(null, null) - builder.append(null, null) - val buf = builder.build() - - val iter = ColumnIterator.newIterator(buf) - - iter.next() - - assert(iter.current == null) - iter.next() - assert(iter.current == null) - iter.next() - assert(iter.current == null) - } - - test("boolean column") { - var builder = new BooleanColumnBuilder - testColumn( - Array[java.lang.Boolean](true, false, true, true, true), - builder, - PrimitiveObjectInspectorFactory.javaBooleanObjectInspector, - PrimitiveObjectInspectorFactory.writableBooleanObjectInspector, - classOf[BooleanColumnIterator]) - assert(builder.stats.min === false) - assert(builder.stats.max === true) - - builder = new BooleanColumnBuilder - testColumn( - Array[java.lang.Boolean](null, false, null, true, true), - builder, - PrimitiveObjectInspectorFactory.javaBooleanObjectInspector, - PrimitiveObjectInspectorFactory.writableBooleanObjectInspector, - classOf[BooleanColumnIterator], - true) - assert(builder.stats.min === false) - assert(builder.stats.max === true) - - builder = new BooleanColumnBuilder - builder.setCompressionSchemes(new RLE) - val a = Array.ofDim[java.lang.Boolean](100) - Range(0,100).foreach { i => - a(i) = if (i < 10) true else if (i <80) false else null - } - testColumn( - a, - builder, - PrimitiveObjectInspectorFactory.javaBooleanObjectInspector, - PrimitiveObjectInspectorFactory.writableBooleanObjectInspector, - classOf[BooleanColumnIterator], - true) - } - - test("byte column") { - var builder = new ByteColumnBuilder - testColumn( - Array[java.lang.Byte](1.toByte, 2.toByte, 15.toByte, 55.toByte, 0.toByte, 40.toByte), - builder, - PrimitiveObjectInspectorFactory.javaByteObjectInspector, - PrimitiveObjectInspectorFactory.writableByteObjectInspector, - classOf[ByteColumnIterator]) - assert(builder.stats.min === 0.toByte) - assert(builder.stats.max === 55.toByte) - - builder = new ByteColumnBuilder - testColumn( - Array[java.lang.Byte](null, 2.toByte, 15.toByte, null, 0.toByte, null), - builder, - PrimitiveObjectInspectorFactory.javaByteObjectInspector, - PrimitiveObjectInspectorFactory.writableByteObjectInspector, - classOf[ByteColumnIterator], - true) - assert(builder.stats.min === 0.toByte) - assert(builder.stats.max === 15.toByte) - - builder = new ByteColumnBuilder - builder.setCompressionSchemes(new RLE) - testColumn( - Array[java.lang.Byte](null, 2.toByte, 2.toByte, null, 4.toByte, 4.toByte,4.toByte,5.toByte), - builder, - PrimitiveObjectInspectorFactory.javaByteObjectInspector, - PrimitiveObjectInspectorFactory.writableByteObjectInspector, - classOf[ByteColumnIterator], - true) - } - - test("short column") { - var builder = new ShortColumnBuilder - testColumn( - Array[java.lang.Short](1.toShort, 2.toShort, -15.toShort, 355.toShort, 0.toShort, 40.toShort), - builder, - PrimitiveObjectInspectorFactory.javaShortObjectInspector, - PrimitiveObjectInspectorFactory.writableShortObjectInspector, - classOf[ShortColumnIterator]) - assert(builder.stats.min === -15.toShort) - assert(builder.stats.max === 355.toShort) - - builder = new ShortColumnBuilder - testColumn( - Array[java.lang.Short](1.toShort, 2.toShort, -15.toShort, null, 0.toShort, null), - builder, - PrimitiveObjectInspectorFactory.javaShortObjectInspector, - PrimitiveObjectInspectorFactory.writableShortObjectInspector, - classOf[ShortColumnIterator], - true) - assert(builder.stats.min === -15.toShort) - assert(builder.stats.max === 2.toShort) - - testColumn( - Array[java.lang.Short](1.toShort, 2.toShort, 2.toShort, null, 1.toShort, 1.toShort), - builder, - PrimitiveObjectInspectorFactory.javaShortObjectInspector, - PrimitiveObjectInspectorFactory.writableShortObjectInspector, - classOf[ShortColumnIterator], - true) - } - - test("int column") { - var builder = new IntColumnBuilder - testColumn( - Array[java.lang.Integer](0, 1, 2, 5, 134, -12, 1, 0, 99, 1), - builder, - PrimitiveObjectInspectorFactory.javaIntObjectInspector, - PrimitiveObjectInspectorFactory.writableIntObjectInspector, - classOf[IntColumnIterator]) - assert(builder.stats.min === -12) - assert(builder.stats.max === 134) - - builder = new IntColumnBuilder - testColumn( - Array[java.lang.Integer](null, 1, 2, 5, 134, -12, null, 0, 99, 1), - builder, - PrimitiveObjectInspectorFactory.javaIntObjectInspector, - PrimitiveObjectInspectorFactory.writableIntObjectInspector, - classOf[IntColumnIterator], - true) - assert(builder.stats.min === -12) - assert(builder.stats.max === 134) - - builder = new IntColumnBuilder - builder.setCompressionSchemes(new RLE) - val a = Array.ofDim[java.lang.Integer](100) - Range(0,100).foreach { i => - a(i) = if (i < 10) 10 else if (i <80) 11 else null - } - - testColumn( - a, - builder, - PrimitiveObjectInspectorFactory.javaIntObjectInspector, - PrimitiveObjectInspectorFactory.writableIntObjectInspector, - classOf[IntColumnIterator], - true) - } - - test("long column") { - var builder = new LongColumnBuilder - testColumn( - Array[java.lang.Long](1L, -345345L, 15L, 0L, 23445456L), - builder, - PrimitiveObjectInspectorFactory.javaLongObjectInspector, - PrimitiveObjectInspectorFactory.writableLongObjectInspector, - classOf[LongColumnIterator]) - assert(builder.stats.min === -345345L) - assert(builder.stats.max === 23445456L) - builder = new LongColumnBuilder - testColumn( - Array[java.lang.Long](null, -345345L, 15L, 0L, null), - builder, - PrimitiveObjectInspectorFactory.javaLongObjectInspector, - PrimitiveObjectInspectorFactory.writableLongObjectInspector, - classOf[LongColumnIterator], - true) - assert(builder.stats.min === -345345L) - assert(builder.stats.max === 15L) - - builder = new LongColumnBuilder - builder.setCompressionSchemes(new RLE) - val a = Array.ofDim[java.lang.Long](100) - Range(0,100).foreach { i => - a(i) = if (i < 10) 10 else if (i <80) 11 else null - } - testColumn( - a, - builder, - PrimitiveObjectInspectorFactory.javaLongObjectInspector, - PrimitiveObjectInspectorFactory.writableLongObjectInspector, - classOf[LongColumnIterator], - true) - } - - test("float column") { - var builder = new FloatColumnBuilder - testColumn( - Array[java.lang.Float](1.1.toFloat, -2.5.toFloat, 20000.toFloat, 0.toFloat, 15.0.toFloat), - builder, - PrimitiveObjectInspectorFactory.javaFloatObjectInspector, - PrimitiveObjectInspectorFactory.writableFloatObjectInspector, - classOf[FloatColumnIterator]) - assert(builder.stats.min === -2.5.toFloat) - assert(builder.stats.max === 20000.toFloat) - builder = new FloatColumnBuilder - testColumn( - Array[java.lang.Float](1.1.toFloat, null, 20000.toFloat, null, 15.0.toFloat), - builder, - PrimitiveObjectInspectorFactory.javaFloatObjectInspector, - PrimitiveObjectInspectorFactory.writableFloatObjectInspector, - classOf[FloatColumnIterator], - true) - assert(builder.stats.min === 1.1.toFloat) - assert(builder.stats.max === 20000.toFloat) - } - - test("double column") { - var builder = new DoubleColumnBuilder - testColumn( - Array[java.lang.Double](1.1, 2.2, -2.5, 20000, 0, 15.0), - builder, - PrimitiveObjectInspectorFactory.javaDoubleObjectInspector, - PrimitiveObjectInspectorFactory.writableDoubleObjectInspector, - classOf[DoubleColumnIterator]) - assert(builder.stats.min === -2.5) - assert(builder.stats.max === 20000) - builder = new DoubleColumnBuilder - testColumn( - Array[java.lang.Double](1.1, 2.2, -2.5, null, 0, 15.0), - builder, - PrimitiveObjectInspectorFactory.javaDoubleObjectInspector, - PrimitiveObjectInspectorFactory.writableDoubleObjectInspector, - classOf[DoubleColumnIterator], - true) - assert(builder.stats.min === -2.5) - assert(builder.stats.max === 15.0) - } - - test("string column") { - var builder = new StringColumnBuilder - testColumn( - Array[Text](new Text("a"), new Text(""), new Text("b"), new Text("Abcdz")), - builder, - PrimitiveObjectInspectorFactory.writableStringObjectInspector, - PrimitiveObjectInspectorFactory.writableStringObjectInspector, - classOf[StringColumnIterator], - false, - (a, b) => (a.equals(b.toString)) - ) - assert(builder.stats.min.toString === "") - assert(builder.stats.max.toString === "b") - - builder = new StringColumnBuilder - testColumn( - Array[Text](new Text("a"), new Text(""), null, new Text("b"), new Text("Abcdz"), null), - builder, - PrimitiveObjectInspectorFactory.writableStringObjectInspector, - PrimitiveObjectInspectorFactory.writableStringObjectInspector, - classOf[StringColumnIterator], - false, - (a, b) => if (a == null) b == null else (a.toString.equals(b.toString)) - ) - assert(builder.stats.min.toString === "") - assert(builder.stats.max.toString === "b") - - builder = new StringColumnBuilder - builder.setCompressionSchemes(new RLE) - testColumn( - Array[Text](new Text("a"), new Text("a"), null, new Text("b"), new Text("b"), new Text("Abcdz")), - builder, - PrimitiveObjectInspectorFactory.writableStringObjectInspector, - PrimitiveObjectInspectorFactory.writableStringObjectInspector, - classOf[StringColumnIterator], - false, - (a, b) => if (a == null) b == null else (a.toString.equals(b.toString)) - ) - } - - test("timestamp column") { - val ts1 = new java.sql.Timestamp(0) - val ts2 = new java.sql.Timestamp(500) - ts2.setNanos(400) - val ts3 = new java.sql.Timestamp(1362561610000L) - - var builder = new TimestampColumnBuilder - testColumn( - Array(ts1, ts2, ts3), - builder, - PrimitiveObjectInspectorFactory.javaTimestampObjectInspector, - PrimitiveObjectInspectorFactory.writableTimestampObjectInspector, - classOf[TimestampColumnIterator], - false, - (a, b) => (a.equals(b)) - ) - assert(builder.stats.min.equals(ts1)) - assert(builder.stats.max.equals(ts3)) - - builder = new TimestampColumnBuilder - testColumn( - Array(ts1, ts2, null, ts3, null), - builder, - PrimitiveObjectInspectorFactory.javaTimestampObjectInspector, - PrimitiveObjectInspectorFactory.writableTimestampObjectInspector, - classOf[TimestampColumnIterator], - true, - (a, b) => (a.equals(b)) - ) - assert(builder.stats.min.equals(ts1)) - assert(builder.stats.max.equals(ts3)) - } - - test("Binary Column") { - val b1 = new BytesWritable() - b1.set(Array[Byte](0,1,2), 0, 3) - - val builder = new BinaryColumnBuilder - testColumn( - Array[BytesWritable](b1), - builder, - PrimitiveObjectInspectorFactory.writableBinaryObjectInspector, - PrimitiveObjectInspectorFactory.writableBinaryObjectInspector, - classOf[BinaryColumnIterator], - false, - compareBinary) - assert(builder.stats.isInstanceOf[ColumnStats.NoOpStats[_]]) - - def compareBinary(x: Object, y: Object): Boolean = { - val xdata = x.asInstanceOf[Array[Byte]] - val ywritable = y.asInstanceOf[BytesWritable] - val ydata = ywritable.getBytes() - val length = ywritable.getLength() - if (length != xdata.length) { - false - } else { - val ydatapruned = new Array[Byte](length) - System.arraycopy(ydata, 0, ydatapruned, 0, length) - java.util.Arrays.equals(xdata, ydatapruned) - } - } - } - - - def testColumn[T, U <: ColumnIterator]( - testData: Array[_ <: Object], - builder: ColumnBuilder[T], - oi: ObjectInspector, - writableOi: AbstractPrimitiveWritableObjectInspector, - iteratorClass: Class[U], - expectEWAHWrapper: Boolean = false, - compareFunc: (Object, Object) => Boolean = (a, b) => a == b) { - - builder.initialize(testData.size, "") - testData.foreach { x => builder.append(x, oi)} - val buf = builder.build() - - def executeOneTest() { - val iter = ColumnIterator.newIterator(buf) - - (0 until testData.size).foreach { i => - iter.next() - val expected = testData(i) - val reality = writableOi.getPrimitiveJavaObject(iter.current) - //println ("at position " + i + " expected " + expected + ", but saw " + reality) - assert((expected == null && reality == null) || compareFunc(reality, expected), - "at position " + i + " expected " + expected + ", but saw " + reality) - } - } - - if (PARALLEL_MODE) { - // parallelize to test concurrency - (1 to 10).par.foreach { parallelIndex => executeOneTest() } - } else { - executeOneTest() - } - } -} - diff --git a/src/main/test/scala/shark/memstore2/ColumnStatsSuite.scala b/src/main/test/scala/shark/memstore2/ColumnStatsSuite.scala deleted file mode 100755 index e1e51402..00000000 --- a/src/main/test/scala/shark/memstore2/ColumnStatsSuite.scala +++ /dev/null @@ -1,232 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark.memstore2 - -import java.sql.Timestamp -import scala.language.implicitConversions - -import org.apache.hadoop.io.Text - -import org.scalatest.FunSuite - -import shark.memstore2.column.ColumnStats - - -class ColumnStatsSuite extends FunSuite { - - test("BooleanColumnStats") { - var c = new ColumnStats.BooleanColumnStats - c.append(false) - assert(c.min == false && c.max == false) - c.append(false) - assert(c.min == false && c.max == false) - c.append(true) - assert(c.min == false && c.max == true) - - c = new ColumnStats.BooleanColumnStats - c.append(true) - assert(c.min == true && c.max == true) - c.append(false) - assert(c.min == false && c.max == true) - assert(c := true) - assert(c := false) - c = new ColumnStats.BooleanColumnStats - c.append(true) - assert(c := true) - assert(!(c := false)) - assert(c :>= false) - assert(!(c :<= false)) - assert(c :>= true) - } - - test("ByteColumnStats") { - val c = new ColumnStats.ByteColumnStats - c.append(0) - assert(c.min == 0 && c.max == 0) - c.append(1) - assert(c.min == 0 && c.max == 1) - c.append(-1) - assert(c.min == -1 && c.max == 1) - c.append(2) - assert(c.min == -1 && c.max == 2) - c.append(-2) - assert(c.min == -2 && c.max == 2) - assert(c := 0.toByte) - assert(c :> 0.toByte) - assert(c :<= -1.toByte) - assert(!(c :<= -3.toByte)) - } - - test("ShortColumnStats") { - val c = new ColumnStats.ShortColumnStats - c.append(0) - assert(c.min == 0 && c.max == 0) - c.append(1) - assert(c.min == 0 && c.max == 1) - c.append(-1) - assert(c.min == -1 && c.max == 1) - c.append(1024) - assert(c.min == -1 && c.max == 1024) - c.append(-1024) - assert(c.min == -1024 && c.max == 1024) - } - - test("IntColumnStats") { - var c = new ColumnStats.IntColumnStats - c.append(0) - assert(c.min == 0 && c.max == 0) - c.append(1) - assert(c.min == 0 && c.max == 1) - c.append(-1) - assert(c.min == -1 && c.max == 1) - c.append(65537) - assert(c.min == -1 && c.max == 65537) - c.append(-65537) - assert(c.min == -65537 && c.max == 65537) - - c = new ColumnStats.IntColumnStats - assert(c.isOrdered && c.isAscending && c.isDescending) - assert(c.maxDelta == 0) - - c = new ColumnStats.IntColumnStats - Array(1).foreach(c.append) - assert(c.isOrdered && c.isAscending && c.isDescending) - assert(c.maxDelta == 0) - - c = new ColumnStats.IntColumnStats - Array(1, 2, 3, 3, 4, 22).foreach(c.append) - assert(c.isOrdered && c.isAscending && !c.isDescending) - assert(c.maxDelta == 18) - - c = new ColumnStats.IntColumnStats - Array(22, 1, 0, -5).foreach(c.append) - assert(c.isOrdered && !c.isAscending && c.isDescending) - assert(c.maxDelta == 21) - - c = new ColumnStats.IntColumnStats - Array(22, 1, 24).foreach(c.append) - assert(!c.isOrdered && !c.isAscending && !c.isDescending) - - c = new ColumnStats.IntColumnStats - Array(22, 1, 24).foreach(c.append) - assert(c := 13 === true) - assert(c :> 13 === true) - assert(c :< 13 === true) - assert(c := 0 === false) - assert(c :> 25 === false) - assert(c :< 1 === false) - } - - test("LongColumnStats") { - val c = new ColumnStats.LongColumnStats - c.append(0) - assert(c.min == 0 && c.max == 0) - c.append(1) - assert(c.min == 0 && c.max == 1) - c.append(-1) - assert(c.min == -1 && c.max == 1) - c.append(Int.MaxValue.toLong + 1L) - assert(c.min == -1 && c.max == Int.MaxValue.toLong + 1L) - c.append(Int.MinValue.toLong - 1L) - assert(c.min == Int.MinValue.toLong - 1L && c.max == Int.MaxValue.toLong + 1L) - assert(c := 0.toLong) - assert(c :> -2.toLong) - assert(c :< 0.toLong) - } - - test("FloatColumnStats") { - val c = new ColumnStats.FloatColumnStats - c.append(0) - assert(c.min == 0 && c.max == 0) - c.append(1) - assert(c.min == 0 && c.max == 1) - c.append(-1) - assert(c.min == -1 && c.max == 1) - c.append(20.5445F) - assert(c.min == -1 && c.max == 20.5445F) - c.append(-20.5445F) - assert(c.min == -20.5445F && c.max == 20.5445F) - assert(c := 20.5F) - assert(c :< 20.6F) - assert(c :> -20.6F) - } - - test("DoubleColumnStats") { - val c = new ColumnStats.DoubleColumnStats - c.append(0) - assert(c.min == 0 && c.max == 0) - c.append(1) - assert(c.min == 0 && c.max == 1) - c.append(-1) - assert(c.min == -1 && c.max == 1) - c.append(20.5445) - assert(c.min == -1 && c.max == 20.5445) - c.append(-20.5445) - assert(c.min == -20.5445 && c.max == 20.5445) - assert(c := 20.5) - assert(!(c :> 20.6)) - assert(c :< 20.6) - } - - test("TimestampColumnStats") { - val c = new ColumnStats.TimestampColumnStats - val ts1 = new Timestamp(1000) - val ts2 = new Timestamp(2000) - val ts3 = new Timestamp(1500) - val ts4 = new Timestamp(2000) - ts4.setNanos(100) - c.append(ts1) - - assert(c.min.equals(ts1) && c.max.equals(ts1)) - c.append(ts2) - assert(c.min.equals(ts1) && c.max.equals(ts2)) - c.append(ts3) - assert(c.min.equals(ts1) && c.max.equals(ts2)) - - - assert(c.min.equals(ts1) && c.max.equals(ts2)) - c.append(ts4) - assert(c.min.equals(ts1) && c.max.equals(ts4)) - - } - - test("StringColumnStats") { - implicit def T(str: String): Text = new Text(str) - val c = new ColumnStats.StringColumnStats - assert(c.min == null && c.max == null) - - assert(!(c :> "test")) - assert(!(c :< "test")) - assert(!(c == "test")) - - c.append("a") - assert(c.min.equals(T("a")) && c.max.equals(T("a"))) - - assert(c.min.equals(T("a")) && c.max.equals(T("a"))) - c.append("b") - assert(c.min.equals(T("a")) && c.max.equals(T("b"))) - c.append("b") - assert(c.min.equals(T("a")) && c.max.equals(T("b"))) - c.append("cccc") - - assert(c.min.equals(T("a")) && c.max.equals(T("cccc"))) - c.append("0987") - assert(c.min.equals(T("0987")) && c.max.equals(T("cccc"))) - - } -} diff --git a/src/main/test/scala/shark/memstore2/TablePartitionSuite.scala b/src/main/test/scala/shark/memstore2/TablePartitionSuite.scala deleted file mode 100755 index fb91027f..00000000 --- a/src/main/test/scala/shark/memstore2/TablePartitionSuite.scala +++ /dev/null @@ -1,121 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark.memstore2 - -import java.nio.ByteBuffer - -import org.scalatest.FunSuite - -import org.apache.spark.SparkConf -import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} - - -class TablePartitionSuite extends FunSuite { - - test("serialize TablePartition backed by non-direct ByteBuffer using Java") { - val col1 = Array[Byte](0, 1, 2) - val col2 = Array[Byte](1, 2, 3) - val tp = new TablePartition(3, Array(ByteBuffer.wrap(col1), ByteBuffer.wrap(col2))) - - val ser = new JavaSerializer(new SparkConf(false)) - val bytes = ser.newInstance().serialize(tp) - val tp1 = ser.newInstance().deserialize[TablePartition](bytes) - assert(tp1.numRows === 3) - assert(tp1.columns(0).remaining() == 3) - assert(tp1.columns(0).get() == 0) - assert(tp1.columns(0).get() == 1) - assert(tp1.columns(0).get() == 2) - assert(tp1.columns(1).remaining() == 3) - assert(tp1.columns(1).get() == 1) - assert(tp1.columns(1).get() == 2) - assert(tp1.columns(1).get() == 3) - } - - test("serialize TablePartition backed by direct ByteBuffer using Java") { - val col1 = ByteBuffer.allocateDirect(3) - col1.put(0.toByte) - col1.put(1.toByte) - col1.put(2.toByte) - col1.rewind() - val col2 = ByteBuffer.allocateDirect(3) - col2.put(1.toByte) - col2.put(2.toByte) - col2.put(3.toByte) - col2.rewind() - val tp = new TablePartition(3, Array(col1, col2)) - - val ser = new JavaSerializer(new SparkConf(false)) - val bytes = ser.newInstance().serialize(tp) - val tp1 = ser.newInstance().deserialize[TablePartition](bytes) - assert(tp1.numRows === 3) - assert(tp1.columns(0).remaining() == 3) - assert(tp1.columns(0).get() == 0) - assert(tp1.columns(0).get() == 1) - assert(tp1.columns(0).get() == 2) - assert(tp1.columns(1).remaining() == 3) - assert(tp1.columns(1).get() == 1) - assert(tp1.columns(1).get() == 2) - assert(tp1.columns(1).get() == 3) - } - - test("serialize TablePartition backed by non-direct ByteBuffer using Kryo") { - val col1 = Array[Byte](0, 1, 2) - val col2 = Array[Byte](1, 2, 3) - val tp = new TablePartition(3, Array(ByteBuffer.wrap(col1), ByteBuffer.wrap(col2))) - - val ser = new KryoSerializer(new SparkConf(false)) - val bytes = ser.newInstance().serialize(tp) - val tp1 = ser.newInstance().deserialize[TablePartition](bytes) - assert(tp1.numRows === 3) - assert(tp1.columns(0).remaining() == 3) - assert(tp1.columns(0).get() == 0) - assert(tp1.columns(0).get() == 1) - assert(tp1.columns(0).get() == 2) - assert(tp1.columns(1).remaining() == 3) - assert(tp1.columns(1).get() == 1) - assert(tp1.columns(1).get() == 2) - assert(tp1.columns(1).get() == 3) - } - - test("serialize TablePartition backed by direct ByteBuffer using Kryo") { - val col1 = ByteBuffer.allocateDirect(3) - col1.put(0.toByte) - col1.put(1.toByte) - col1.put(2.toByte) - col1.rewind() - val col2 = ByteBuffer.allocateDirect(3) - col2.put(1.toByte) - col2.put(2.toByte) - col2.put(3.toByte) - col2.rewind() - val tp = new TablePartition(3, Array(col1, col2)) - - val ser = new KryoSerializer(new SparkConf(false)) - val bytes = ser.newInstance().serialize(tp) - val tp1 = ser.newInstance().deserialize[TablePartition](bytes) - assert(tp1.numRows === 3) - assert(tp1.columns(0).remaining() == 3) - assert(tp1.columns(0).get() == 0) - assert(tp1.columns(0).get() == 1) - assert(tp1.columns(0).get() == 2) - assert(tp1.columns(1).remaining() == 3) - assert(tp1.columns(1).get() == 1) - assert(tp1.columns(1).get() == 2) - assert(tp1.columns(1).get() == 3) - } -} diff --git a/src/main/test/scala/shark/memstore2/column/ColumnTypeSuite.scala b/src/main/test/scala/shark/memstore2/column/ColumnTypeSuite.scala deleted file mode 100755 index 404a01b1..00000000 --- a/src/main/test/scala/shark/memstore2/column/ColumnTypeSuite.scala +++ /dev/null @@ -1,121 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark.memstore2.column - -import java.nio.ByteBuffer - -import org.apache.hadoop.io.IntWritable -import org.apache.hadoop.io.LongWritable -import org.apache.hadoop.hive.serde2.io._ - -import org.scalatest.FunSuite - -class ColumnTypeSuite extends FunSuite { - - test("Int") { - assert(INT.defaultSize == 4) - var buffer = ByteBuffer.allocate(32) - var a: Seq[Int] = Array[Int](35, 67, 899, 4569001) - a.foreach {i => buffer.putInt(i)} - buffer.rewind() - a.foreach {i => - val v = INT.extract(buffer) - assert(v == i) - } - buffer = ByteBuffer.allocate(32) - a = Range(0, 4) - a.foreach { i => - INT.append(i, buffer) - } - buffer.rewind() - a.foreach { i => assert(buffer.getInt() == i)} - - buffer = ByteBuffer.allocate(32) - a =Range(0,4) - a.foreach { i => buffer.putInt(i)} - buffer.rewind() - val writable = new IntWritable() - a.foreach { i => - INT.extractInto(buffer, writable) - assert(writable.get == i) - } - - } - - test("Short") { - assert(SHORT.defaultSize == 2) - assert(SHORT.actualSize(8) == 2) - var buffer = ByteBuffer.allocate(32) - var a = Array[Short](35, 67, 87, 45) - a.foreach {i => buffer.putShort(i)} - buffer.rewind() - a.foreach {i => - val v = SHORT.extract(buffer) - assert(v == i) - } - - buffer = ByteBuffer.allocate(32) - a = Array[Short](0,1,2,3) - a.foreach { i => - SHORT.append(i, buffer) - } - buffer.rewind() - a.foreach { i => assert(buffer.getShort() == i)} - - buffer = ByteBuffer.allocate(32) - a =Array[Short](0,1,2,3) - a.foreach { i => buffer.putShort(i)} - buffer.rewind() - val writable = new ShortWritable() - a.foreach { i => - SHORT.extractInto(buffer, writable) - assert(writable.get == i) - } - } - - test("Long") { - assert(LONG.defaultSize == 8) - assert(LONG.actualSize(45L) == 8) - var buffer = ByteBuffer.allocate(64) - var a = Array[Long](35L, 67L, 8799000880L, 45000999090L) - a.foreach {i => buffer.putLong(i)} - buffer.rewind() - a.foreach {i => - val v = LONG.extract(buffer) - assert(v == i) - } - - buffer = ByteBuffer.allocate(32) - a = Array[Long](0,1,2,3) - a.foreach { i => - LONG.append(i, buffer) - } - buffer.rewind() - a.foreach { i => assert(buffer.getLong() == i)} - - buffer = ByteBuffer.allocate(32) - a =Array[Long](0,1,2,3) - a.foreach { i => buffer.putLong(i)} - buffer.rewind() - val writable = new LongWritable() - a.foreach { i => - LONG.extractInto(buffer, writable) - assert(writable.get == i) - } - } -} diff --git a/src/main/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala b/src/main/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala deleted file mode 100755 index d891ec13..00000000 --- a/src/main/test/scala/shark/memstore2/column/CompressedColumnIteratorSuite.scala +++ /dev/null @@ -1,243 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark.memstore2.column - -import java.nio.ByteBuffer -import java.nio.ByteOrder -import org.scalatest.FunSuite - -import org.apache.hadoop.io.Text -import org.apache.hadoop.hive.serde2.objectinspector.primitive._ -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector - -import shark.memstore2.column.Implicits._ - -class CompressedColumnIteratorSuite extends FunSuite { - - /** - * Generic tester across types and encodings. The function applies the given compression - * algorithm on the given sequence of values, and test whether the resulting iterator gives - * the same sequence of values. - * - * If we expect the compression algorithm to not compress the data, we should set the - * shouldNotCompress flag to true. This way, it doesn't actually create a compressed buffer, - * but simply tests the compression ratio returned by the algorithm is >= 1.0. - */ - def testList[T, W]( - l: Seq[T], - t: ColumnType[T, _], - algo: CompressionAlgorithm, - expectedCompressedSize: Long, - shouldNotCompress: Boolean = false) - { - val b = ByteBuffer.allocate(1024 + (3 * 40 * l.size)) - b.order(ByteOrder.nativeOrder()) - b.putInt(t.typeID) - l.foreach { item => - t.append(item, b) - algo.gatherStatsForCompressibility(item, t.asInstanceOf[ColumnType[Any, _]]) - } - b.limit(b.position()) - b.rewind() - - info("compressed size: %d, uncompressed size: %d, compression ratio %f".format( - algo.compressedSize, algo.uncompressedSize, algo.compressionRatio)) - - info("expected compressed size: %d".format(expectedCompressedSize)) - assert(algo.compressedSize === expectedCompressedSize) - - if (shouldNotCompress) { - assert(algo.compressionRatio >= 1.0) - } else { - val compressedBuffer = algo.compress(b, t) - val iter = new TestIterator(compressedBuffer, compressedBuffer.getInt()) - - val oi: ObjectInspector = t match { - case BOOLEAN => PrimitiveObjectInspectorFactory.writableBooleanObjectInspector - case BYTE => PrimitiveObjectInspectorFactory.writableByteObjectInspector - case SHORT => PrimitiveObjectInspectorFactory.writableShortObjectInspector - case INT => PrimitiveObjectInspectorFactory.writableIntObjectInspector - case LONG => PrimitiveObjectInspectorFactory.writableLongObjectInspector - case STRING => PrimitiveObjectInspectorFactory.writableStringObjectInspector - case _ => throw new UnsupportedOperationException("Unsupported compression type " + t) - } - - l.foreach { x => - assert(iter.hasNext) - iter.next() - assert(t.get(iter.current, oi) === x) - } - - // Make sure we reach the end of the iterator. - assert(!iter.hasNext) - } - } - - test("RLE Boolean") { - // 3 runs: (1+4)*3 - val bools = Seq(true, true, false, true, true, true, true, true, true, true, true, true) - testList(bools, BOOLEAN, new RLE, 15) - } - - test("RLE Byte") { - // 3 runs: (1+4)*3 - testList(Seq[Byte](10, 10, 10, 10, 10, 10, 10, 10, 10, 20, 10), BYTE, new RLE, 15) - } - - test("RLE Short") { - // 3 runs: (2+4)*3 - testList(Seq[Short](10, 10, 10, 20000, 20000, 20000, 500, 500, 500, 500), SHORT, new RLE, 18) - } - - test("RLE Int") { - // 3 runs: (4+4)*3 - testList(Seq[Int](1000000, 1000000, 1000000, 1000000, 900000, 99), INT, new RLE, 24) - } - - test("RLE Long") { - // 2 runs: (8+4)*3 - val longs = Seq[Long](2147483649L, 2147483649L, 2147483649L, 2147483649L, 500L, 500L, 500L) - testList(longs, LONG, new RLE, 24) - } - - test("RLE String") { - // 3 runs: (4+4+4) + (4+1+4) + (4+1+4) = 30 - val strs: Seq[Text] = Seq("abcd", "abcd", "abcd", "e", "e", "!", "!").map(s => new Text(s)) - testList(strs, STRING, new RLE, 30) - } - - test("Dictionary Encoded Int") { - // dict len + 3 distinct values + 7 values = 4 + 3*4 + 7*2 = 30 - val ints = Seq[Int](1000000, 1000000, 99, 1000000, 1000000, 900000, 99) - testList(ints, INT, new DictionaryEncoding, 30) - } - - test("Dictionary Encoded Long") { - // dict len + 2 distinct values + 7 values = 4 + 2*8 + 7*2 = 34 - val longs = Seq[Long](2147483649L, 2147483649L, 2147483649L, 2147483649L, 500L, 500L, 500L) - testList(longs, LONG, new DictionaryEncoding, 34) - } - - test("Dictionary Encoded String") { - // dict len + 3 distinct values + 8 values = 4 + (4+4) + (4+1) + (4+1) + 8*2 = - val strs: Seq[Text] = Seq("abcd", "abcd", "abcd", "e", "e", "e", "!", "!").map(s => new Text(s)) - testList(strs, STRING, new DictionaryEncoding, 38, shouldNotCompress = false) - } - - test("Dictionary Encoding at limit of unique values") { - val ints = Range(0, Short.MaxValue - 1).flatMap(i => Iterator(i, i, i)) - val expectedLen = 4 + (Short.MaxValue - 1) * 4 + 2 * (Short.MaxValue - 1) * 3 - testList(ints, INT, new DictionaryEncoding, expectedLen) - } - - test("Dictionary Encoding - should not compress") { - val ints = Range(0, Short.MaxValue.toInt) - testList(ints, INT, new DictionaryEncoding, Int.MaxValue, shouldNotCompress = true) - } - - test("RLE - should not compress") { - val ints = Range(0, Short.MaxValue.toInt + 1) - val expectedLen = (Short.MaxValue.toInt + 1) * (4 + 4) - testList(ints, INT, new RLE, expectedLen, shouldNotCompress = true) - } - - test("BooleanBitSet Boolean (shorter)") { - // 1 Long worth of Booleans, in addition to the length field: 4+8 - val bools = Seq(true, true, false, false) - testList(bools, BOOLEAN, new BooleanBitSetCompression, 4+8) - } - - test("BooleanBitSet Boolean (longer)") { - // 2 Longs worth of Booleans, in addition to the length field: 4+8+8 - val bools = Seq(true, true, false, false, true, true, false, false,true, true, false, false,true, true, false, false, - true, true, false, false,true, true, false, false, true, true, false, false,true, true, false, false, - true, true, false, false,true, true, false, false, true, true, false, false,true, true, false, false, - true, true, false, false,true, true, false, false, true, true, false, false,true, true, false, false, - true, true, false, false,true, true, false, false, true, true, false, false,true, true, false, false) - testList(bools, BOOLEAN, new BooleanBitSetCompression, 4+8+8) - } - - test("BooleanBitSet Boolean should not compress - compression ratio > 1") { - // 1 Long worth of Booleans, in addtion to the length field: 4+8 - val bools = Seq(true, false) - testList(bools, BOOLEAN, new BooleanBitSetCompression, 4+8, shouldNotCompress = true) - } - - test("IntDeltaEncoding") { - // base 5 + 4 small diffs + newBase 5 = 14 - val ints = Seq[Int](1000000, 1000001, 1000002, 1000003, 1000004, 5) - testList(ints, INT, new IntDeltaEncoding, 5 + 4 + 5) - - val ints2 = Seq[Int](1000000, 1000001, 1000000, 1000004, 1000001, 5) - testList(ints2, INT, new IntDeltaEncoding, 5 + 4 + 5) - - testList(List(0, 62), INT, new IntDeltaEncoding, 1 + 4 + 1) - testList(List(0, 63), INT, new IntDeltaEncoding, 1 + 4 + 1) - testList(List(0, 64), INT, new IntDeltaEncoding, 1 + 4 + 1) - testList(List(0, 63, 64), INT, new IntDeltaEncoding, 1 + 4 + 1 + 1) - testList(List(0, 128, -125), INT, new IntDeltaEncoding, 1 + 4 + 1 + 4 + 1 + 4) - - testList(List(0, 12400, 12600, 100, 228), INT, new IntDeltaEncoding, 5 * 5) - testList(Range(-4, 0), INT, new IntDeltaEncoding, 1 + 4 + 3) - - val ints3 = Range(0, Byte.MaxValue.toInt - 1) - testList(ints3, INT, new IntDeltaEncoding, 1 + 4 + 125) - - val ints4 = Range(Byte.MinValue.toInt + 2, 0) - testList(ints4, INT, new IntDeltaEncoding, 1 + 4 + 125) - } - - test("LongDeltaEncoding") { - // base 9 + 3 small deltas + newBase 9 + 2 small deltas = 23 - val longs = Seq[Long](2147483649L, 2147483649L, 2147483649L, 2147483649L, 500L, 500L, 500L) - testList(longs, LONG, new LongDeltaEncoding, 23) - } - - test("int delta encoding boundary condition") { - // 127 deltas are fine, while 128 are not. - var skips = Range(0, 1000).map { x => -127 * x } - testList(skips, INT, new IntDeltaEncoding, 1 + 4 + 999) - - skips = Range(0, 1000).map { x => 127 * x } - testList(skips, INT, new IntDeltaEncoding, 1 + 4 + 999) - - skips = Range(0, 1000).map { x => 128 * x } - testList(skips, INT, new IntDeltaEncoding, 1 + 4 + (5 * 999)) - - skips = Range(0, 1000).map { x => -128 * x } - testList(skips, INT, new IntDeltaEncoding, 1 + 4 + (5 * 999)) - } - - test("long delta encoding boundary condition") { - // 127 deltas are fine, while 128 are not. - var skips = Range(0, 1000).map { x => (-127 * x).toLong } - testList(skips, LONG, new LongDeltaEncoding, 1 + 8 + 999) - - skips = Range(0, 1000).map { x => (127 * x).toLong } - testList(skips, LONG, new LongDeltaEncoding, 1 + 8 + 999) - - skips = Range(0, 1000).map { x => (128 * x).toLong } - testList(skips, LONG, new LongDeltaEncoding, 1 + 8 + (9 * 999)) - - skips = Range(0, 1000).map { x => (-128 * x).toLong } - testList(skips, LONG, new LongDeltaEncoding, 1 + 8 + (9 * 999)) - } -} - -class TestIterator(val buffer: ByteBuffer, val columnType: ColumnType[_,_]) - extends CompressedColumnIterator diff --git a/src/main/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala b/src/main/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala deleted file mode 100755 index 7f7a118e..00000000 --- a/src/main/test/scala/shark/memstore2/column/CompressionAlgorithmSuite.scala +++ /dev/null @@ -1,366 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark.memstore2.column - -import java.nio.{ByteBuffer, ByteOrder} - -import scala.collection.mutable.HashMap - -import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory -import org.apache.hadoop.io.Text - -import org.scalatest.FunSuite - -import shark.memstore2.column.ColumnStats._ - -class CompressionAlgorithmSuite extends FunSuite { - - // TODO: clean these tests. - - test("CompressedColumnBuilder using RLE") { - - class TestColumnBuilder(val stats: ColumnStats[Int], val t: ColumnType[Int,_]) - extends CompressedColumnBuilder[Int] { - override def shouldApply(scheme: CompressionAlgorithm) = true - } - - val b = new TestColumnBuilder(new NoOpStats, INT) - b.setCompressionSchemes(new RLE) - b.initialize(100) - val oi = PrimitiveObjectInspectorFactory.javaIntObjectInspector - b.append(123.asInstanceOf[Object], oi) - b.append(123.asInstanceOf[Object], oi) - b.append(56.asInstanceOf[Object], oi) - b.append(56.asInstanceOf[Object], oi) - val compressedBuffer = b.build() - assert(compressedBuffer.getInt() == INT.typeID) - assert(compressedBuffer.getInt() == RLECompressionType.typeID) - - assert(compressedBuffer.getInt() == 123) - assert(compressedBuffer.getInt() == 2) - assert(compressedBuffer.getInt() == 56) - assert(compressedBuffer.getInt() == 2) - assert(!compressedBuffer.hasRemaining) - } - - test("RLE Strings") { - val b = ByteBuffer.allocate(1024) - b.order(ByteOrder.nativeOrder()) - b.putInt(STRING.typeID) - val rle = new RLE() - Seq[Text](new Text("abc"), new Text("abc"), new Text("efg"), new Text("abc")).foreach { text => - STRING.append(text, b) - rle.gatherStatsForCompressibility(text, STRING) - } - b.limit(b.position()) - b.rewind() - val compressedBuffer = rle.compress(b, STRING) - assert(compressedBuffer.getInt() == STRING.typeID) - assert(compressedBuffer.getInt() == RLECompressionType.typeID) - assert(STRING.extract(compressedBuffer).equals(new Text("abc"))) - assert(compressedBuffer.getInt() == 2) - assert(STRING.extract(compressedBuffer).equals(new Text("efg"))) - assert(compressedBuffer.getInt() == 1) - assert(STRING.extract(compressedBuffer).equals(new Text("abc"))) - assert(compressedBuffer.getInt() == 1) - assert(!compressedBuffer.hasRemaining) - } - - test("RLE int with run length 1") { - val b = ByteBuffer.allocate(16) - b.order(ByteOrder.nativeOrder()) - b.putInt(INT.typeID) - b.putInt(123) - b.putInt(56) - b.limit(b.position()) - b.rewind() - val rle = new RLE() - rle.gatherStatsForCompressibility(123, INT) - rle.gatherStatsForCompressibility(56, INT) - val compressedBuffer = rle.compress(b, INT) - assert(compressedBuffer.getInt() == INT.typeID) - assert(compressedBuffer.getInt() == RLECompressionType.typeID) - assert(compressedBuffer.getInt() == 123) - assert(compressedBuffer.getInt() == 1) - assert(compressedBuffer.getInt() == 56) - assert(compressedBuffer.getInt() == 1) - assert(!compressedBuffer.hasRemaining) - } - - test("RLE int single run") { - val b = ByteBuffer.allocate(4008) - b.order(ByteOrder.nativeOrder()) - b.putInt(INT.typeID) - val rle = new RLE() - Range(0, 1000).foreach { x => - b.putInt(6) - rle.gatherStatsForCompressibility(6, INT) - } - b.limit(b.position()) - b.rewind() - val compressedBuffer = rle.compress(b, INT) - assert(compressedBuffer.getInt() == INT.typeID) - assert(compressedBuffer.getInt() == RLECompressionType.typeID) - assert(compressedBuffer.getInt() == 6) - assert(compressedBuffer.getInt() == 1000) - assert(!compressedBuffer.hasRemaining) - } - - test("RLE long single run") { - val b = ByteBuffer.allocate(8008) - b.order(ByteOrder.nativeOrder()) - b.putInt(LONG.typeID) - val rle = new RLE() - Range(0, 1000).foreach { x => - b.putLong(Long.MaxValue - 6) - rle.gatherStatsForCompressibility(Long.MaxValue - 6, LONG) - } - b.limit(b.position()) - b.rewind() - val compressedBuffer = rle.compress(b, LONG) - assert(compressedBuffer.getInt() == LONG.typeID) - assert(compressedBuffer.getInt() == RLECompressionType.typeID) - assert(compressedBuffer.getLong() == Long.MaxValue - 6) - assert(compressedBuffer.getInt() == 1000) - assert(!compressedBuffer.hasRemaining) - } - - test("RLE int 3 runs") { - val b = ByteBuffer.allocate(4008) - b.order(ByteOrder.nativeOrder()) - b.putInt(INT.typeID) - val items = Array[Int](10, 20, 40) - val rle = new RLE() - - Range(0, 1000).foreach { x => - val v = if (x < 100) items(0) else if (x < 500) items(1) else items(2) - b.putInt(v) - rle.gatherStatsForCompressibility(v, INT) - } - b.limit(b.position()) - b.rewind() - val compressedBuffer = rle.compress(b, INT) - assert(compressedBuffer.getInt() == INT.typeID) - assert(compressedBuffer.getInt() == RLECompressionType.typeID) - assert(compressedBuffer.getInt() == 10) - assert(compressedBuffer.getInt() == 100) - assert(compressedBuffer.getInt() == 20) - assert(compressedBuffer.getInt() == 400) - assert(compressedBuffer.getInt() == 40) - assert(compressedBuffer.getInt() == 500) - assert(!compressedBuffer.hasRemaining) - } - - test("RLE int single long run") { - val b = ByteBuffer.allocate(4000008) - b.order(ByteOrder.nativeOrder()) - b.putInt(INT.typeID) - val rle = new RLE() - - Range(0, 1000000).foreach { x => - b.putInt(6) - rle.gatherStatsForCompressibility(6, INT) - } - b.limit(b.position()) - b.rewind() - val compressedBuffer = rle.compress(b, INT) - assert(compressedBuffer.getInt() == RLECompressionType.typeID) - assert(compressedBuffer.getInt() == INT.typeID) - assert(compressedBuffer.getInt() == 6) - assert(compressedBuffer.getInt() == 1000000) - assert(!compressedBuffer.hasRemaining) - } - - test("IntDeltaEncoding") { - val b = ByteBuffer.allocate(1024) - b.order(ByteOrder.nativeOrder()) - b.putInt(INT.typeID) - - val bde = new IntDeltaEncoding - - val x = 1 - b.putInt(x) - bde.gatherStatsForCompressibility(x, INT) - - val y = x + 40000 - b.putInt(y) - bde.gatherStatsForCompressibility(y, INT) - - val z = y + 1 - b.putInt(z) - bde.gatherStatsForCompressibility(z, INT) - - b.limit(b.position()) - b.rewind() - val compressedBuffer = bde.compress(b, INT) - assert(compressedBuffer.getInt() == INT.typeID) - assert(compressedBuffer.getInt() == IntDeltaCompressionType.typeID) - - compressedBuffer.get() // first flagByte - assert(INT.extract(compressedBuffer).equals(x)) - - compressedBuffer.get() // second flagByte - assert(INT.extract(compressedBuffer).equals(y)) - - val seven: Byte = compressedBuffer.get() // third flagByte - assert(seven === 1.toByte) - - assert(!compressedBuffer.hasRemaining) - } - - test("LongDeltaEncoding") { - val b = ByteBuffer.allocate(10024) - b.order(ByteOrder.nativeOrder()) - b.putInt(LONG.typeID) - - val bde = new LongDeltaEncoding - - val x: Long = 1 - b.putLong(x) - bde.gatherStatsForCompressibility(x, LONG) - - val y: Long = x + 40000 - b.putLong(y) - bde.gatherStatsForCompressibility(y, LONG) - - val z: Long = y + 1 - b.putLong(z) - bde.gatherStatsForCompressibility(z, LONG) - - b.limit(b.position()) - b.rewind() - val compressedBuffer = bde.compress(b, LONG) - assert(compressedBuffer.getInt() === LONG.typeID) - assert(compressedBuffer.getInt() === LongDeltaCompressionType.typeID) - - compressedBuffer.get() // first flagByte - assert(LONG.extract(compressedBuffer).equals(x)) - - compressedBuffer.get() // second flagByte - assert(LONG.extract(compressedBuffer).equals(y)) - - val seven: Byte = compressedBuffer.get() // third flagByte - assert(seven === 1.toByte) - - assert(!compressedBuffer.hasRemaining) - } - - test("Dictionary Encoding") { - - def testList[T]( - l: Seq[T], - u: ColumnType[T, _], - expectedDictSize: Int, - compareFunc: (T, T) => Boolean = (a: T, b: T) => a == b) { - - val b = ByteBuffer.allocate(1024 + (3*40*l.size)) - b.order(ByteOrder.nativeOrder()) - b.putInt(u.typeID) - val de = new DictionaryEncoding() - l.foreach { item => - assert(de.supportsType(u)) - u.append(item, b) - de.gatherStatsForCompressibility(item, u) - } - b.limit(b.position()) - b.rewind() - val compressedBuffer = de.compress(b, u) - assert(compressedBuffer.getInt() === u.typeID) - assert(compressedBuffer.getInt() === DictionaryCompressionType.typeID) - assert(compressedBuffer.getInt() === expectedDictSize) //dictionary size - val dictionary = new HashMap[Short, T]() - var count = 0 - while (count < expectedDictSize) { - val v = u.extract(compressedBuffer) - dictionary.put(dictionary.size.toShort, u.clone(v)) - count += 1 - } - assert(dictionary.get(0).get.equals(l(0))) - assert(dictionary.get(1).get.equals(l(2))) - l.foreach { x => - val y = dictionary.get(compressedBuffer.getShort()).get - assert(compareFunc(y, x)) - } - } - - val iList = Array[Int](10, 10, 20, 10) - val lList = iList.map { i => Long.MaxValue - i.toLong } - val sList = iList.map { i => new Text(i.toString) } - - testList(iList, INT, 2) - testList(lList, LONG, 2) - testList(sList, STRING, 2, (a: Text, b: Text) => a.hashCode == b.hashCode) - - // test at limit of unique values - val alternating = Range(0, Short.MaxValue-1, 1).flatMap { s => List(1, s) } - val longList = List.concat(iList, alternating, iList) - assert(longList.size === (8 + 2*(Short.MaxValue-1))) - testList(longList, INT, Short.MaxValue - 1) - } - - test("Uncompressed text") { - val b = new StringColumnBuilder - b.initialize(0) - val oi = PrimitiveObjectInspectorFactory.javaStringObjectInspector - - val lines = Array[String]( - "lar deposits. blithely final packages cajole. regular waters are final requests.", - "hs use ironic, even requests. s", - "ges. thinly even pinto beans ca", - "ly final courts cajole furiously final excuse", - "uickly special accounts cajole carefully blithely close requests. carefully final" - ) - lines.foreach { line => - b.append(line, oi) - } - val newBuffer = b.build() - assert(newBuffer.getInt() === 0) // null count - assert(newBuffer.getInt() === STRING.typeID) - assert(newBuffer.getInt() === DefaultCompressionType.typeID) - } - - test("BooleanBitSet encoding") { - val bbs = new BooleanBitSetCompression() - val b = ByteBuffer.allocate(4 + 64 + 2) - b.order(ByteOrder.nativeOrder()) - b.putInt(BOOLEAN.typeID) - for(_ <- 1 to 5) { - b.put(0.toByte) - b.put(1.toByte) - bbs.gatherStatsForCompressibility(false, BOOLEAN) - bbs.gatherStatsForCompressibility(true, BOOLEAN) - } - for(_ <- 1 to 54) { - b.put(0.toByte) - bbs.gatherStatsForCompressibility(false, BOOLEAN) - } - b.put(0.toByte) - b.put(1.toByte) - bbs.gatherStatsForCompressibility(false, BOOLEAN) - bbs.gatherStatsForCompressibility(true, BOOLEAN) - b.limit(b.position()) - b.rewind() - val compressedBuffer = bbs.compress(b, BOOLEAN) - assert(compressedBuffer.getInt() === BOOLEAN.typeID) - assert(compressedBuffer.getInt() === BooleanBitSetCompressionType.typeID) - assert(compressedBuffer.getInt() === 64 + 2) - assert(compressedBuffer.getLong() === 682) - assert(compressedBuffer.getLong() === 2) - assert(!compressedBuffer.hasRemaining) - } -} diff --git a/src/main/test/scala/shark/memstore2/column/NullableColumnBuilderSuite.scala b/src/main/test/scala/shark/memstore2/column/NullableColumnBuilderSuite.scala deleted file mode 100755 index 2208569e..00000000 --- a/src/main/test/scala/shark/memstore2/column/NullableColumnBuilderSuite.scala +++ /dev/null @@ -1,145 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark.memstore2.column - -import org.apache.hadoop.io.Text -import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory - -import org.scalatest.FunSuite - -class NullableColumnBuilderSuite extends FunSuite { - - test("Empty column") { - val c = new IntColumnBuilder() - c.initialize(4) - val b = c.build() - // # of nulls - assert(b.getInt() === 0) - // column type - assert(b.getInt() === INT.typeID) - assert(b.getInt() === DefaultCompressionType.typeID) - assert(!b.hasRemaining) - } - - test("Buffer size auto growth") { - val c = new StringColumnBuilder() - c.initialize(4) - val oi = PrimitiveObjectInspectorFactory.writableStringObjectInspector - c.append(new Text("a"), oi) - c.append(null, oi) - c.append(new Text("b"), oi) - c.append(null, oi) - c.append(new Text("abc"), oi) - c.append(null, oi) - c.append(null, oi) - c.append(new Text("efg"), oi) - val b = c.build() - b.position(4 + 4 * 4) - val colType = b.getInt() - assert(colType === STRING.typeID) - } - - test("Null Strings") { - val c = new StringColumnBuilder() - c.initialize(4) - val oi = PrimitiveObjectInspectorFactory.writableStringObjectInspector - c.append(new Text("a"), oi) - c.append(null, oi) - c.append(new Text("b"), oi) - c.append(null, oi) - val b = c.build() - - // Number of nulls - assert(b.getInt() === 2) - - // First null position is 1, and then 3 - assert(b.getInt() === 1) - assert(b.getInt() === 3) - - // Column data type - assert(b.getInt() === STRING.typeID) - - // Compression type - assert(b.getInt() === DefaultCompressionType.typeID) - - // Data - assert(b.getInt() === 1) - assert(b.get() === 97) - assert(b.getInt() === 1) - assert(b.get() === 98) - } - - test("Null Ints") { - val c = new IntColumnBuilder() - c.initialize(4) - val oi = PrimitiveObjectInspectorFactory.javaIntObjectInspector - c.append(123.asInstanceOf[Object], oi) - c.append(null, oi) - c.append(null, oi) - c.append(56.asInstanceOf[Object], oi) - val b = c.build() - - // # of nulls and null positions - assert(b.getInt() === 2) - assert(b.getInt() === 1) - assert(b.getInt() === 2) - - // non nulls - assert(b.getInt() === INT.typeID) - assert(b.getInt() === DefaultCompressionType.typeID) - assert(b.getInt() === 123) - } - - test("Nullable Ints 2") { - val c = new IntColumnBuilder() - c.initialize(4) - val oi = PrimitiveObjectInspectorFactory.javaIntObjectInspector - Range(1, 1000).foreach { x => - c.append(x.asInstanceOf[Object], oi) - } - val b = c.build() - // null count - assert(b.getInt() === 0) - // column type - assert(b.getInt() === INT.typeID) - // compression type - assert(b.getInt() === DefaultCompressionType.typeID) - } - - test("Null Longs") { - val c = new LongColumnBuilder() - c.initialize(4) - val oi = PrimitiveObjectInspectorFactory.javaLongObjectInspector - c.append(123L.asInstanceOf[Object], oi) - c.append(null, oi) - c.append(null, oi) - c.append(56L.asInstanceOf[Object], oi) - val b = c.build() - - // # of nulls and null positions - assert(b.getInt() === 2) - assert(b.getInt() === 1) - assert(b.getInt() === 2) - - // non-nulls - assert(b.getInt() === LONG.typeID) - assert(b.getInt() === DefaultCompressionType.typeID) - assert(b.getLong() === 123L) - } - -} diff --git a/src/main/test/scala/shark/memstore2/column/NullableColumnIteratorSuite.scala b/src/main/test/scala/shark/memstore2/column/NullableColumnIteratorSuite.scala deleted file mode 100755 index d4784fa5..00000000 --- a/src/main/test/scala/shark/memstore2/column/NullableColumnIteratorSuite.scala +++ /dev/null @@ -1,118 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark.memstore2.column - -import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory -import org.apache.hadoop.io.Text -import org.apache.hadoop.io.IntWritable - -import org.scalatest.FunSuite - - -class NullableColumnIteratorSuite extends FunSuite { - - test("String Growth") { - val c = new StringColumnBuilder - c.initialize(4, "") - val oi = PrimitiveObjectInspectorFactory.writableStringObjectInspector - - val a = Array[Text]( - new Text("a"), null, - new Text("b"), null, - new Text("abc"), null, - null, null, new Text("efg") - ) - a.foreach { - t => c.append(t, oi) - } - val b = c.build() - val i = ColumnIterator.newIterator(b) - Range(0, a.length).foreach { x => - if (x > 0) assert(i.hasNext) - i.next() - val v = i.current - if (a(x) == null) { - assert(v == null) - } else { - assert(v.toString == a(x).toString) - } - } - assert(!i.hasNext) - } - - test("Iterate Strings") { - val c = new StringColumnBuilder - c.initialize(4, "") - val oi = PrimitiveObjectInspectorFactory.writableStringObjectInspector - - c.append(new Text("a"), oi) - c.append(new Text(""), oi) - c.append(null, oi) - c.append(new Text("b"), oi) - c.append(new Text("Abcdz"), oi) - c.append(null, oi) - val b = c.build() - val i = ColumnIterator.newIterator(b) - i.next() - assert(i.current.toString() == "a") - i.next() - assert(i.current.toString() == "") - i.next() - assert(i.current == null) - i.next() - assert(i.current.toString() == "b") - i.next() - assert(i.current.toString() == "Abcdz") - i.next() - assert(i.current == null) - assert(false === i.hasNext) - } - - test("Iterate Ints") { - def testList(l: Seq[AnyRef]) { - val c = new IntColumnBuilder - c.initialize(l.size, "") - val oi = PrimitiveObjectInspectorFactory.javaIntObjectInspector - - l.foreach { item => - if (item == null) { - c.append(null, oi) - } else { - c.append(item.asInstanceOf[Object], oi) - } - } - - val b = c.build() - val i = ColumnIterator.newIterator(b) - - l.foreach { x => - i.next() - if (x == null) { - assert(i.current === x) - } else { - assert(i.current.asInstanceOf[IntWritable].get === x) - } - } - assert(false === i.hasNext) - } - - testList(List(null, null, 123.asInstanceOf[AnyRef])) - testList(List(123.asInstanceOf[AnyRef], 4.asInstanceOf[AnyRef], null)) - testList(List(null)) - } -} diff --git a/src/main/test/scala/shark/util/BloomFilterSuite.scala b/src/main/test/scala/shark/util/BloomFilterSuite.scala deleted file mode 100755 index 8c1507d0..00000000 --- a/src/main/test/scala/shark/util/BloomFilterSuite.scala +++ /dev/null @@ -1,35 +0,0 @@ -package shark.util - -import org.scalatest.FunSuite - -class BloomFilterSuite extends FunSuite{ - - test("Integer") { - val bf = new BloomFilter(0.03, 1000000) - Range(0, 1000000).foreach { - i => bf.add(i) - } - assert(bf.contains(333)) - assert(bf.contains(678)) - assert(!bf.contains(1200000)) - } - - test("Integer FP") { - val bf = new BloomFilter(0.03,1000) - Range(0,700).foreach { - i => bf.add(i) - } - assert(bf.contains(333)) - assert(bf.contains(678)) - //is the fraction of false positives in line with what we expect ? - val e = Range(0, 100).map { - i => bf.contains(i*10) - } - val s = e.groupBy(x => x).map(x => (x._1, x._2.size)) - val t = s(true) - val f = s(false) - assert(f > 25 && f < 35) - assert(t < 75 && t > 65) - // expect false positive to be < 3 % and no false negatives - } -} \ No newline at end of file diff --git a/src/main/test/tests_fail.txt b/src/main/test/tests_fail.txt deleted file mode 100755 index 5e66a75c..00000000 --- a/src/main/test/tests_fail.txt +++ /dev/null @@ -1,232 +0,0 @@ -testCliDriver_allcolref_in_udf -testCliDriver_alter_merge_orc -testCliDriver_alter_merge_orc_stats -testCliDriver_alter_numbuckets_partitioned_table -testCliDriver_alter_numbuckets_partitioned_table2 -testCliDriver_alter_partition_coltype -testCliDriver_alter_table_serde2 -testCliDriver_alter_view_as_select -testCliDriver_archive_multi -testCliDriver_auto_join25 -testCliDriver_auto_join29 -testCliDriver_auto_join32 -testCliDriver_auto_join7 -testCliDriver_auto_join_reordering_values -testCliDriver_auto_smb_mapjoin_14 -testCliDriver_auto_sortmerge_join_1 -testCliDriver_auto_sortmerge_join_10 -testCliDriver_auto_sortmerge_join_2 -testCliDriver_auto_sortmerge_join_3 -testCliDriver_auto_sortmerge_join_4 -testCliDriver_auto_sortmerge_join_5 -testCliDriver_auto_sortmerge_join_6 -testCliDriver_auto_sortmerge_join_7 -testCliDriver_auto_sortmerge_join_8 -testCliDriver_auto_sortmerge_join_9 -testCliDriver_avro_joins -testCliDriver_ba_table_udfs -testCliDriver_bucket2 -testCliDriver_bucket3 -testCliDriver_bucket_groupby -testCliDriver_bucketcontext_1 -testCliDriver_bucketcontext_2 -testCliDriver_bucketcontext_3 -testCliDriver_bucketcontext_4 -testCliDriver_bucketcontext_5 -testCliDriver_bucketcontext_6 -testCliDriver_bucketcontext_7 -testCliDriver_bucketcontext_8 -testCliDriver_bucketizedhiveinputformat_auto -testCliDriver_bucketmapjoin1 -testCliDriver_bucketmapjoin13 -testCliDriver_bucketmapjoin2 -testCliDriver_bucketmapjoin3 -testCliDriver_bucketmapjoin4 -testCliDriver_bucketmapjoin5 -testCliDriver_column_access_stats -testCliDriver_columnstats_partlvl -testCliDriver_columnstats_tbllvl -testCliDriver_combine2_hadoop20 -testCliDriver_combine3 -testCliDriver_concatenate_inherit_table_location -testCliDriver_ctas_hadoop20 -testCliDriver_ctas_uses_database_location -testCliDriver_custom_input_output_format -testCliDriver_escape1 -testCliDriver_escape2 -testCliDriver_exim_04_evolved_parts -testCliDriver_exim_11_managed_external -testCliDriver_exim_12_external_location -testCliDriver_exim_13_managed_location -testCliDriver_exim_14_managed_location_over_existing -testCliDriver_exim_15_external_part -testCliDriver_exim_16_part_external -testCliDriver_exim_17_part_managed -testCliDriver_exim_18_part_external -testCliDriver_exim_19_00_part_external_location -testCliDriver_exim_19_part_external_location -testCliDriver_exim_20_part_managed_location -testCliDriver_filter_join_breaktask -testCliDriver_filter_join_breaktask2 -testCliDriver_groupby10 -testCliDriver_groupby11 -testCliDriver_groupby7_map_multi_single_reducer -testCliDriver_groupby7_noskew_multi_single_reducer -testCliDriver_groupby8 -testCliDriver_groupby8_map -testCliDriver_groupby8_map_skew -testCliDriver_groupby8_noskew -testCliDriver_groupby9 -testCliDriver_groupby_complex_types_multi_single_reducer -testCliDriver_groupby_multi_insert_common_distinct -testCliDriver_groupby_multi_single_reducer -testCliDriver_groupby_multi_single_reducer2 -testCliDriver_groupby_multi_single_reducer3 -testCliDriver_groupby_position -testCliDriver_groupby_sort_1 -testCliDriver_groupby_sort_6 -testCliDriver_groupby_sort_skew_1 -testCliDriver_hook_context_cs -testCliDriver_index_auto_update -testCliDriver_index_bitmap_rc -testCliDriver_infer_bucket_sort -testCliDriver_infer_bucket_sort_convert_join -testCliDriver_infer_bucket_sort_grouping_operators -testCliDriver_infer_bucket_sort_multi_insert -testCliDriver_input3_limit -testCliDriver_input42 -testCliDriver_input_part8 -testCliDriver_join32_lessSize -testCliDriver_join7 -testCliDriver_join_1to1 -testCliDriver_join_filters -testCliDriver_join_filters_overlap -testCliDriver_join_map_ppr -testCliDriver_join_nullsafe -testCliDriver_join_reorder -testCliDriver_join_reorder2 -testCliDriver_join_reorder3 -testCliDriver_join_vc -testCliDriver_leadlag -testCliDriver_leadlag_queries -testCliDriver_mapjoin_hook -testCliDriver_merge3 -testCliDriver_merge4 -testCliDriver_merge_dynamic_partition -testCliDriver_merge_dynamic_partition2 -testCliDriver_merge_dynamic_partition3 -testCliDriver_metadataonly1 -testCliDriver_multi_insert -testCliDriver_multi_insert_gby -testCliDriver_multi_insert_move_tasks_share_dependencies -testCliDriver_nestedvirtual -testCliDriver_nonmr_fetch -testCliDriver_null_column -testCliDriver_orc_create -testCliDriver_orc_empty_files -testCliDriver_overridden_confs -testCliDriver_parallel -testCliDriver_partInit -testCliDriver_partition_wise_fileformat14 -testCliDriver_pcr -testCliDriver_ppd_union_view -testCliDriver_ptf -testCliDriver_ptf_decimal -testCliDriver_ptf_matchpath -testCliDriver_ptf_rcfile -testCliDriver_ptf_register_tblfn -testCliDriver_ptf_seqfile -testCliDriver_query_result_fileformat -testCliDriver_rand_partitionpruner2 -testCliDriver_reduce_deduplicate_extended -testCliDriver_rename_partition_location -testCliDriver_rename_table_location -testCliDriver_sample10 -testCliDriver_sample6 -testCliDriver_select_transform_hint -testCliDriver_semicolon -testCliDriver_semijoin -testCliDriver_show_functions -testCliDriver_skewjoin -testCliDriver_skewjoinopt1 -testCliDriver_skewjoinopt10 -testCliDriver_skewjoinopt11 -testCliDriver_skewjoinopt12 -testCliDriver_skewjoinopt14 -testCliDriver_skewjoinopt15 -testCliDriver_skewjoinopt16 -testCliDriver_skewjoinopt17 -testCliDriver_skewjoinopt19 -testCliDriver_skewjoinopt2 -testCliDriver_skewjoinopt20 -testCliDriver_skewjoinopt3 -testCliDriver_skewjoinopt4 -testCliDriver_skewjoinopt5 -testCliDriver_skewjoinopt6 -testCliDriver_skewjoinopt7 -testCliDriver_skewjoinopt8 -testCliDriver_smb_mapjoin9 -testCliDriver_smb_mapjoin_1 -testCliDriver_smb_mapjoin_11 -testCliDriver_smb_mapjoin_12 -testCliDriver_smb_mapjoin_13 -testCliDriver_smb_mapjoin_14 -testCliDriver_smb_mapjoin_15 -testCliDriver_smb_mapjoin_16 -testCliDriver_smb_mapjoin_17 -testCliDriver_smb_mapjoin_18 -testCliDriver_smb_mapjoin_19 -testCliDriver_smb_mapjoin_2 -testCliDriver_smb_mapjoin_20 -testCliDriver_smb_mapjoin_21 -testCliDriver_smb_mapjoin_22 -testCliDriver_smb_mapjoin_3 -testCliDriver_smb_mapjoin_4 -testCliDriver_smb_mapjoin_5 -testCliDriver_smb_mapjoin_6 -testCliDriver_smb_mapjoin_7 -testCliDriver_sort_merge_join_desc_1 -testCliDriver_sort_merge_join_desc_2 -testCliDriver_sort_merge_join_desc_3 -testCliDriver_sort_merge_join_desc_5 -testCliDriver_sort_merge_join_desc_6 -testCliDriver_sort_merge_join_desc_7 -testCliDriver_stats1 -testCliDriver_stats10 -testCliDriver_stats11 -testCliDriver_stats12 -testCliDriver_stats13 -testCliDriver_stats14 -testCliDriver_stats15 -testCliDriver_stats16 -testCliDriver_stats18 -testCliDriver_stats19 -testCliDriver_stats2 -testCliDriver_stats20 -testCliDriver_stats3 -testCliDriver_stats4 -testCliDriver_stats5 -testCliDriver_stats6 -testCliDriver_stats7 -testCliDriver_stats8 -testCliDriver_stats9 -testCliDriver_stats_empty_partition -testCliDriver_stats_noscan_1 -testCliDriver_stats_noscan_2 -testCliDriver_stats_partscan_1 -testCliDriver_table_access_keys_stats -testCliDriver_udf_bitmap_and -testCliDriver_udf_bitmap_empty -testCliDriver_udf_bitmap_or -testCliDriver_udf_context_aware -testCliDriver_udf_in_file -testCliDriver_union17 -testCliDriver_union22 -testCliDriver_union24 -testCliDriver_union31 -testCliDriver_union_null -testCliDriver_uniquejoin -testCliDriver_virtual_column -testCliDriver_windowing -testCliDriver_windowing_expressions -testCliDriver_windowing_multipartitioning diff --git a/src/main/test/tests_pass.txt b/src/main/test/tests_pass.txt deleted file mode 100755 index 389c3779..00000000 --- a/src/main/test/tests_pass.txt +++ /dev/null @@ -1,925 +0,0 @@ -testCliDriver_add_part_exist -testCliDriver_add_partition_no_whitelist -testCliDriver_add_partition_with_whitelist -testCliDriver_alias_casted_column -testCliDriver_alter1 -testCliDriver_alter2 -testCliDriver_alter3 -testCliDriver_alter4 -testCliDriver_alter5 -testCliDriver_alter_concatenate_indexed_table -testCliDriver_alter_index -testCliDriver_alter_merge -testCliDriver_alter_merge_2 -testCliDriver_alter_merge_orc2 -testCliDriver_alter_merge_stats -testCliDriver_alter_partition_clusterby_sortby -testCliDriver_alter_partition_format_loc -testCliDriver_alter_partition_protect_mode -testCliDriver_alter_partition_with_whitelist -testCliDriver_alter_rename_partition -testCliDriver_alter_rename_partition_authorization -testCliDriver_alter_skewed_table -testCliDriver_alter_table_not_sorted -testCliDriver_alter_table_serde -testCliDriver_alter_view_rename -testCliDriver_ambiguous_col -testCliDriver_authorization_1 -testCliDriver_authorization_2 -testCliDriver_authorization_3 -testCliDriver_authorization_4 -testCliDriver_authorization_5 -testCliDriver_authorization_6 -testCliDriver_authorization_7 -testCliDriver_autogen_colalias -testCliDriver_auto_join0 -testCliDriver_auto_join1 -testCliDriver_auto_join10 -testCliDriver_auto_join11 -testCliDriver_auto_join12 -testCliDriver_auto_join13 -testCliDriver_auto_join14 -testCliDriver_auto_join14_hadoop20 -testCliDriver_auto_join15 -testCliDriver_auto_join16 -testCliDriver_auto_join17 -testCliDriver_auto_join18 -testCliDriver_auto_join18_multi_distinct -testCliDriver_auto_join19 -testCliDriver_auto_join2 -testCliDriver_auto_join20 -testCliDriver_auto_join21 -testCliDriver_auto_join22 -testCliDriver_auto_join23 -testCliDriver_auto_join24 -testCliDriver_auto_join26 -testCliDriver_auto_join27 -testCliDriver_auto_join28 -testCliDriver_auto_join3 -testCliDriver_auto_join30 -testCliDriver_auto_join31 -testCliDriver_auto_join4 -testCliDriver_auto_join5 -testCliDriver_auto_join6 -testCliDriver_auto_join8 -testCliDriver_auto_join9 -testCliDriver_auto_join_filters -testCliDriver_auto_join_nulls -testCliDriver_auto_sortmerge_join_11 -testCliDriver_auto_sortmerge_join_12 -testCliDriver_avro_change_schema -testCliDriver_avro_compression_enabled -testCliDriver_avro_evolved_schemas -testCliDriver_avro_nullable_fields -testCliDriver_avro_sanity_test -testCliDriver_avro_schema_error_message -testCliDriver_avro_schema_literal -testCliDriver_ba_table1 -testCliDriver_ba_table2 -testCliDriver_ba_table3 -testCliDriver_ba_table_union -testCliDriver_binary_constant -testCliDriver_binary_output_format -testCliDriver_binarysortable_1 -testCliDriver_binary_table_bincolserde -testCliDriver_binary_table_colserde -testCliDriver_bucket1 -testCliDriver_bucket_map_join_1 -testCliDriver_bucketmapjoin10 -testCliDriver_bucketmapjoin11 -testCliDriver_bucketmapjoin12 -testCliDriver_bucket_map_join_2 -testCliDriver_bucketmapjoin8 -testCliDriver_bucketmapjoin9 -testCliDriver_bucketmapjoin_negative -testCliDriver_bucketmapjoin_negative2 -testCliDriver_bucketmapjoin_negative3 -testCliDriver_case_sensitivity -testCliDriver_cast1 -testCliDriver_cluster -testCliDriver_columnarserde_create_shortcut -testCliDriver_combine1 -testCliDriver_compute_stats_binary -testCliDriver_compute_stats_boolean -testCliDriver_compute_stats_double -testCliDriver_compute_stats_empty_table -testCliDriver_compute_stats_long -testCliDriver_compute_stats_string -testCliDriver_constant_prop -testCliDriver_convert_enum_to_string -testCliDriver_count -testCliDriver_cp_mj_rc -testCliDriver_create_1 -testCliDriver_create_alter_list_bucketing_table1 -testCliDriver_create_big_view -testCliDriver_create_default_prop -testCliDriver_create_escape -testCliDriver_create_genericudaf -testCliDriver_create_genericudf -testCliDriver_create_insert_outputformat -testCliDriver_create_like -testCliDriver_create_like2 -testCliDriver_create_like_tbl_props -testCliDriver_create_like_view -testCliDriver_create_merge_compressed -testCliDriver_create_nested_type -testCliDriver_create_or_replace_view -testCliDriver_create_skewed_table1 -testCliDriver_create_struct_table -testCliDriver_create_udaf -testCliDriver_create_union_table -testCliDriver_create_view -testCliDriver_create_view_partitioned -testCliDriver_cross_join -testCliDriver_ct_case_insensitive -testCliDriver_database -testCliDriver_database_drop -testCliDriver_database_location -testCliDriver_database_properties -testCliDriver_ddltime -testCliDriver_decimal_1 -testCliDriver_decimal_2 -testCliDriver_decimal_3 -testCliDriver_decimal_4 -testCliDriver_decimal_precision -testCliDriver_decimal_serde -testCliDriver_decimal_udf -testCliDriver_default_partition_name -testCliDriver_delimiter -testCliDriver_desc_non_existent_tbl -testCliDriver_describe_comment_indent -testCliDriver_describe_database_json -testCliDriver_describe_formatted_view_partitioned -testCliDriver_describe_formatted_view_partitioned_json -testCliDriver_describe_pretty -testCliDriver_describe_syntax -testCliDriver_describe_table -testCliDriver_describe_table_json -testCliDriver_describe_xpath -testCliDriver_diff_part_input_formats -testCliDriver_disable_file_format_check -testCliDriver_driverhook -testCliDriver_drop_database_removes_partition_dirs -testCliDriver_drop_function -testCliDriver_drop_index -testCliDriver_drop_index_removes_partition_dirs -testCliDriver_drop_multi_partitions -testCliDriver_drop_partitions_filter -testCliDriver_drop_partitions_filter2 -testCliDriver_drop_partitions_filter3 -testCliDriver_drop_partitions_ignore_protection -testCliDriver_drop_table -testCliDriver_drop_table2 -testCliDriver_drop_table_removes_partition_dirs -testCliDriver_drop_udf -testCliDriver_drop_view -testCliDriver_enforce_order -testCliDriver_escape_clusterby1 -testCliDriver_escape_distributeby1 -testCliDriver_escape_orderby1 -testCliDriver_escape_sortby1 -testCliDriver_exim_00_nonpart_empty -testCliDriver_exim_01_nonpart -testCliDriver_exim_02_00_part_empty -testCliDriver_exim_02_part -testCliDriver_exim_03_nonpart_over_compat -testCliDriver_exim_04_all_part -testCliDriver_exim_05_some_part -testCliDriver_exim_06_one_part -testCliDriver_exim_07_all_part_over_nonoverlap -testCliDriver_exim_08_nonpart_rename -testCliDriver_exim_09_part_spec_nonoverlap -testCliDriver_exim_10_external_managed -testCliDriver_exim_21_export_authsuccess -testCliDriver_exim_22_import_exist_authsuccess -testCliDriver_exim_23_import_part_authsuccess -testCliDriver_exim_24_import_nonexist_authsuccess -testCliDriver_explain_dependency -testCliDriver_explain_dependency2 -testCliDriver_explode_null -testCliDriver_fileformat_mix -testCliDriver_fileformat_sequencefile -testCliDriver_fileformat_text -testCliDriver_global_limit -testCliDriver_groupby1 -testCliDriver_groupby1_limit -testCliDriver_groupby1_map -testCliDriver_groupby1_map_nomap -testCliDriver_groupby1_map_skew -testCliDriver_groupby1_noskew -testCliDriver_groupby2_limit -testCliDriver_groupby2_map -testCliDriver_groupby2_map_multi_distinct -testCliDriver_groupby2_map_skew -testCliDriver_groupby2_map_skew_multi_distinct -testCliDriver_groupby2_multi_distinct -testCliDriver_groupby2_noskew -testCliDriver_groupby2_noskew_multi_distinct -testCliDriver_groupby3 -testCliDriver_groupby3_map -testCliDriver_groupby3_map_multi_distinct -testCliDriver_groupby3_map_skew -testCliDriver_groupby3_noskew -testCliDriver_groupby3_noskew_multi_distinct -testCliDriver_groupby4 -testCliDriver_groupby4_map -testCliDriver_groupby4_map_skew -testCliDriver_groupby4_noskew -testCliDriver_groupby5 -testCliDriver_groupby5_map -testCliDriver_groupby5_map_skew -testCliDriver_groupby5_noskew -testCliDriver_groupby6 -testCliDriver_groupby6_map -testCliDriver_groupby6_map_skew -testCliDriver_groupby6_noskew -testCliDriver_groupby7 -testCliDriver_groupby7_map -testCliDriver_groupby7_map_skew -testCliDriver_groupby7_noskew -testCliDriver_groupby_bigdata -testCliDriver_groupby_complex_types -testCliDriver_groupby_cube1 -testCliDriver_groupby_distinct_samekey -testCliDriver_groupby_grouping_id1 -testCliDriver_groupby_grouping_id2 -testCliDriver_groupby_grouping_sets1 -testCliDriver_groupby_grouping_sets2 -testCliDriver_groupby_grouping_sets3 -testCliDriver_groupby_grouping_sets4 -testCliDriver_groupby_grouping_sets5 -testCliDriver_groupby_map_ppr -testCliDriver_groupby_map_ppr_multi_distinct -testCliDriver_groupby_mutli_insert_common_distinct -testCliDriver_groupby_neg_float -testCliDriver_groupby_ppd -testCliDriver_groupby_ppr -testCliDriver_groupby_ppr_multi_distinct -testCliDriver_groupby_rollup1 -testCliDriver_groupby_sort_10 -testCliDriver_groupby_sort_2 -testCliDriver_groupby_sort_3 -testCliDriver_groupby_sort_4 -testCliDriver_groupby_sort_5 -testCliDriver_groupby_sort_7 -testCliDriver_groupby_sort_8 -testCliDriver_groupby_sort_9 -testCliDriver_groupby_sort_test_1 -testCliDriver_having -testCliDriver_hook_order -testCliDriver_implicit_cast1 -testCliDriver_index_auth -testCliDriver_index_auto -testCliDriver_index_auto_empty -testCliDriver_index_auto_file_format -testCliDriver_index_auto_multiple -testCliDriver_index_auto_mult_tables -testCliDriver_index_auto_mult_tables_compact -testCliDriver_index_auto_partitioned -testCliDriver_index_auto_self_join -testCliDriver_index_auto_unused -testCliDriver_index_bitmap -testCliDriver_index_bitmap1 -testCliDriver_index_bitmap2 -testCliDriver_index_bitmap3 -testCliDriver_index_bitmap_auto -testCliDriver_index_bitmap_auto_partitioned -testCliDriver_index_bitmap_compression -testCliDriver_index_compact -testCliDriver_index_compact_1 -testCliDriver_index_compact_2 -testCliDriver_index_compact_3 -testCliDriver_index_compact_binary_search -testCliDriver_index_compression -testCliDriver_index_creation -testCliDriver_index_stale -testCliDriver_index_stale_partitioned -testCliDriver_infer_const_type -testCliDriver_init_file -testCliDriver_innerjoin -testCliDriver_inoutdriver -testCliDriver_input -testCliDriver_input0 -testCliDriver_input1 -testCliDriver_input10 -testCliDriver_input11 -testCliDriver_input11_limit -testCliDriver_input12_hadoop20 -testCliDriver_input13 -testCliDriver_input14 -testCliDriver_input14_limit -testCliDriver_input15 -testCliDriver_input16 -testCliDriver_input17 -testCliDriver_input18 -testCliDriver_input19 -testCliDriver_input1_limit -testCliDriver_input2 -testCliDriver_input20 -testCliDriver_input21 -testCliDriver_input22 -testCliDriver_input23 -testCliDriver_input24 -testCliDriver_input25 -testCliDriver_input26 -testCliDriver_input28 -testCliDriver_input2_limit -testCliDriver_input3 -testCliDriver_input30 -testCliDriver_input31 -testCliDriver_input32 -testCliDriver_input33 -testCliDriver_input34 -testCliDriver_input35 -testCliDriver_input36 -testCliDriver_input37 -testCliDriver_input38 -testCliDriver_input39_hadoop20 -testCliDriver_input4 -testCliDriver_input40 -testCliDriver_input41 -testCliDriver_input43 -testCliDriver_input44 -testCliDriver_input45 -testCliDriver_input46 -testCliDriver_input49 -testCliDriver_input4_cb_delim -testCliDriver_input4_limit -testCliDriver_input5 -testCliDriver_input6 -testCliDriver_input7 -testCliDriver_input8 -testCliDriver_input9 -testCliDriver_input_columnarserde -testCliDriver_inputddl1 -testCliDriver_inputddl2 -testCliDriver_inputddl3 -testCliDriver_inputddl4 -testCliDriver_inputddl5 -testCliDriver_inputddl6 -testCliDriver_inputddl7 -testCliDriver_inputddl8 -testCliDriver_input_dfs -testCliDriver_input_dynamicserde -testCliDriver_input_lazyserde -testCliDriver_input_limit -testCliDriver_input_part0 -testCliDriver_input_part1 -testCliDriver_input_part10 -testCliDriver_input_part2 -testCliDriver_input_part3 -testCliDriver_input_part4 -testCliDriver_input_part5 -testCliDriver_input_part6 -testCliDriver_input_part7 -testCliDriver_input_part9 -testCliDriver_input_testsequencefile -testCliDriver_input_testxpath -testCliDriver_input_testxpath2 -testCliDriver_input_testxpath3 -testCliDriver_input_testxpath4 -testCliDriver_insert1 -testCliDriver_insert1_overwrite_partitions -testCliDriver_insert2_overwrite_partitions -testCliDriver_insert_compressed -testCliDriver_insertexternal1 -testCliDriver_insert_into1 -testCliDriver_insert_into2 -testCliDriver_insert_into3 -testCliDriver_insert_into4 -testCliDriver_insert_into5 -testCliDriver_insert_into6 -testCliDriver_insert_overwrite_local_directory_1 -testCliDriver_join0 -testCliDriver_join10 -testCliDriver_join11 -testCliDriver_join12 -testCliDriver_join13 -testCliDriver_join14 -testCliDriver_join14_hadoop20 -testCliDriver_join15 -testCliDriver_join16 -testCliDriver_join17 -testCliDriver_join18 -testCliDriver_join18_multi_distinct -testCliDriver_join19 -testCliDriver_join2 -testCliDriver_join20 -testCliDriver_join21 -testCliDriver_join22 -testCliDriver_join23 -testCliDriver_join24 -testCliDriver_join25 -testCliDriver_join26 -testCliDriver_join27 -testCliDriver_join28 -testCliDriver_join29 -testCliDriver_join3 -testCliDriver_join30 -testCliDriver_join31 -testCliDriver_join32 -testCliDriver_join33 -testCliDriver_join34 -testCliDriver_join35 -testCliDriver_join36 -testCliDriver_join37 -testCliDriver_join38 -testCliDriver_join39 -testCliDriver_join4 -testCliDriver_join40 -testCliDriver_join41 -testCliDriver_join5 -testCliDriver_join6 -testCliDriver_join8 -testCliDriver_join9 -testCliDriver_join_array -testCliDriver_join_casesensitive -testCliDriver_join_empty -testCliDriver_join_hive_626 -testCliDriver_join_nulls -testCliDriver_join_rc -testCliDriver_join_reorder4 -testCliDriver_join_star -testCliDriver_join_thrift -testCliDriver_join_view -testCliDriver_keyword_1 -testCliDriver_lateral_view -testCliDriver_lateral_view_cp -testCliDriver_lateral_view_ppd -testCliDriver_leftsemijoin -testCliDriver_lineage1 -testCliDriver_literal_decimal -testCliDriver_literal_double -testCliDriver_literal_ints -testCliDriver_literal_string -testCliDriver_load_binary_data -testCliDriver_load_dyn_part1 -testCliDriver_load_dyn_part10 -testCliDriver_load_dyn_part11 -testCliDriver_load_dyn_part12 -testCliDriver_load_dyn_part13 -testCliDriver_load_dyn_part14 -testCliDriver_load_dyn_part15 -testCliDriver_load_dyn_part2 -testCliDriver_load_dyn_part3 -testCliDriver_load_dyn_part4 -testCliDriver_load_dyn_part5 -testCliDriver_load_dyn_part6 -testCliDriver_load_dyn_part7 -testCliDriver_load_dyn_part8 -testCliDriver_load_dyn_part9 -testCliDriver_load_exist_part_authsuccess -testCliDriver_load_fs -testCliDriver_load_nonpart_authsuccess -testCliDriver_load_overwrite -testCliDriver_loadpart1 -testCliDriver_load_part_authsuccess -testCliDriver_lock1 -testCliDriver_lock2 -testCliDriver_lock3 -testCliDriver_lock4 -testCliDriver_louter_join_ppr -testCliDriver_mapjoin1 -testCliDriver_mapjoin_distinct -testCliDriver_mapjoin_filter_on_outerjoin -testCliDriver_mapjoin_mapjoin -testCliDriver_mapjoin_subquery -testCliDriver_mapjoin_subquery2 -testCliDriver_mapjoin_test_outer -testCliDriver_mapreduce1 -testCliDriver_mapreduce2 -testCliDriver_mapreduce3 -testCliDriver_mapreduce4 -testCliDriver_mapreduce5 -testCliDriver_mapreduce6 -testCliDriver_mapreduce7 -testCliDriver_mapreduce8 -testCliDriver_merge1 -testCliDriver_merge2 -testCliDriver_merge_dynamic_partition4 -testCliDriver_merge_dynamic_partition5 -testCliDriver_mergejoins -testCliDriver_mergejoins_mixed -testCliDriver_metadata_export_drop -testCliDriver_mi -testCliDriver_misc_json -testCliDriver_multigroupby_singlemr -testCliDriver_multi_join_union -testCliDriver_multiMapJoin1 -testCliDriver_multi_sahooks -testCliDriver_newline -testCliDriver_noalias_subq1 -testCliDriver_no_hooks -testCliDriver_nomore_ambiguous_table_col -testCliDriver_nonblock_op_deduplicate -testCliDriver_nonreserved_keywords_input37 -testCliDriver_nonreserved_keywords_insert_into1 -testCliDriver_notable_alias1 -testCliDriver_notable_alias2 -testCliDriver_null_cast -testCliDriver_nullgroup -testCliDriver_nullgroup2 -testCliDriver_nullgroup3 -testCliDriver_nullgroup4 -testCliDriver_nullgroup4_multi_distinct -testCliDriver_nullgroup5 -testCliDriver_nullinput -testCliDriver_nullinput2 -testCliDriver_nullscript -testCliDriver_num_op_type_conv -testCliDriver_ops_comparison -testCliDriver_orc_createas1 -testCliDriver_orc_diff_part_cols -testCliDriver_orc_empty_strings -testCliDriver_orc_ends_with_nulls -testCliDriver_order -testCliDriver_order2 -testCliDriver_outer_join_ppr -testCliDriver_parenthesis_star_by -testCliDriver_partcols1 -testCliDriver_part_inherit_tbl_props -testCliDriver_part_inherit_tbl_props_empty -testCliDriver_part_inherit_tbl_props_with_star -testCliDriver_partition_decode_name -testCliDriver_partition_schema1 -testCliDriver_partition_serde_format -testCliDriver_partitions_json -testCliDriver_partition_special_char -testCliDriver_partition_vs_table_metadata -testCliDriver_partition_wise_fileformat -testCliDriver_partition_wise_fileformat10 -testCliDriver_partition_wise_fileformat11 -testCliDriver_partition_wise_fileformat12 -testCliDriver_partition_wise_fileformat13 -testCliDriver_partition_wise_fileformat15 -testCliDriver_partition_wise_fileformat16 -testCliDriver_partition_wise_fileformat2 -testCliDriver_partition_wise_fileformat3 -testCliDriver_partition_wise_fileformat4 -testCliDriver_partition_wise_fileformat5 -testCliDriver_partition_wise_fileformat6 -testCliDriver_partition_wise_fileformat7 -testCliDriver_partition_wise_fileformat8 -testCliDriver_partition_wise_fileformat9 -testCliDriver_plan_json -testCliDriver_ppd1 -testCliDriver_ppd2 -testCliDriver_ppd_clusterby -testCliDriver_ppd_constant_expr -testCliDriver_ppd_constant_where -testCliDriver_ppd_gby -testCliDriver_ppd_gby2 -testCliDriver_ppd_gby_join -testCliDriver_ppd_join -testCliDriver_ppd_join2 -testCliDriver_ppd_join3 -testCliDriver_ppd_join_filter -testCliDriver_ppd_multi_insert -testCliDriver_ppd_outer_join1 -testCliDriver_ppd_outer_join2 -testCliDriver_ppd_outer_join3 -testCliDriver_ppd_outer_join4 -testCliDriver_ppd_outer_join5 -testCliDriver_ppd_random -testCliDriver_ppd_repeated_alias -testCliDriver_ppd_transform -testCliDriver_ppd_udf_case -testCliDriver_ppd_udf_col -testCliDriver_ppd_union -testCliDriver_ppr_allchildsarenull -testCliDriver_ppr_pushdown -testCliDriver_ppr_pushdown2 -testCliDriver_ppr_pushdown3 -testCliDriver_print_header -testCliDriver_progress_1 -testCliDriver_protectmode -testCliDriver_protectmode2 -testCliDriver_ptf_general_queries -testCliDriver_ql_rewrite_gbtoidx -testCliDriver_query_properties -testCliDriver_query_with_semi -testCliDriver_quote1 -testCliDriver_quote2 -testCliDriver_rand_partitionpruner1 -testCliDriver_rand_partitionpruner3 -testCliDriver_rcfile_bigdata -testCliDriver_rcfile_columnar -testCliDriver_rcfile_createas1 -testCliDriver_rcfile_default_format -testCliDriver_rcfile_lazydecompress -testCliDriver_rcfile_null_value -testCliDriver_rcfile_toleratecorruptions -testCliDriver_rcfile_union -testCliDriver_reduce_deduplicate_exclude_gby -testCliDriver_reduce_deduplicate_exclude_join -testCliDriver_reducesink_dedup -testCliDriver_regex_col -testCliDriver_regexp_extract -testCliDriver_rename_column -testCliDriver_rename_external_partition_location -testCliDriver_repair -testCliDriver_reset_conf -testCliDriver_router_join_ppr -testCliDriver_sample1 -testCliDriver_sample2 -testCliDriver_sample3 -testCliDriver_sample4 -testCliDriver_sample5 -testCliDriver_sample7 -testCliDriver_sample8 -testCliDriver_sample9 -testCliDriver_sample_islocalmode_hook_hadoop20 -testCliDriver_script_env_var1 -testCliDriver_script_env_var2 -testCliDriver_script_pipe -testCliDriver_select_as_omitted -testCliDriver_select_unquote_and -testCliDriver_select_unquote_not -testCliDriver_select_unquote_or -testCliDriver_serde_regex -testCliDriver_serde_reported_schema -testCliDriver_set_processor_namespaces -testCliDriver_set_variable_sub -testCliDriver_show_columns -testCliDriver_show_create_table_alter -testCliDriver_show_create_table_db_table -testCliDriver_show_create_table_delimited -testCliDriver_show_create_table_partitioned -testCliDriver_show_create_table_serde -testCliDriver_show_create_table_view -testCliDriver_show_describe_func_quotes -testCliDriver_show_indexes_edge_cases -testCliDriver_show_indexes_syntax -testCliDriver_show_partitions -testCliDriver_showparts -testCliDriver_show_tables -testCliDriver_show_tablestatus -testCliDriver_show_tblproperties -testCliDriver_shutdown -testCliDriver_skewjoinopt13 -testCliDriver_skewjoinopt18 -testCliDriver_skewjoinopt9 -testCliDriver_skewjoin_union_remove_1 -testCliDriver_skewjoin_union_remove_2 -testCliDriver_smb_mapjoin_10 -testCliDriver_sort -testCliDriver_sort_merge_join_desc_4 -testCliDriver_source -testCliDriver_stats0 -testCliDriver_stats_aggregator_error_1 -testCliDriver_stats_empty_dyn_part -testCliDriver_stats_publisher_error_1 -testCliDriver_str_to_map -testCliDriver_subq -testCliDriver_subq2 -testCliDriver_symlink_text_input_format -testCliDriver_tablename_with_select -testCliDriver_test_boolean_whereclause -testCliDriver_timestamp_1 -testCliDriver_timestamp_2 -testCliDriver_timestamp_3 -testCliDriver_timestamp_comparison -testCliDriver_timestamp_lazy -testCliDriver_timestamp_null -testCliDriver_timestamp_udf -testCliDriver_touch -testCliDriver_transform1 -testCliDriver_transform2 -testCliDriver_transform_ppr1 -testCliDriver_transform_ppr2 -testCliDriver_truncate_table -testCliDriver_type_cast_1 -testCliDriver_type_widening -testCliDriver_udaf_collect_set -testCliDriver_udaf_context_ngrams -testCliDriver_udaf_corr -testCliDriver_udaf_covar_pop -testCliDriver_udaf_covar_samp -testCliDriver_udaf_histogram_numeric -testCliDriver_udaf_ngrams -testCliDriver_udaf_number_format -testCliDriver_udaf_percentile_approx -testCliDriver_udf1 -testCliDriver_udf_10_trims -testCliDriver_udf2 -testCliDriver_udf3 -testCliDriver_udf4 -testCliDriver_udf5 -testCliDriver_udf6 -testCliDriver_udf7 -testCliDriver_udf8 -testCliDriver_udf9 -testCliDriver_udf_abs -testCliDriver_udf_acos -testCliDriver_udf_add -testCliDriver_udf_array -testCliDriver_udf_array_contains -testCliDriver_udf_ascii -testCliDriver_udf_asin -testCliDriver_udf_atan -testCliDriver_udf_avg -testCliDriver_udf_between -testCliDriver_udf_bigint -testCliDriver_udf_bin -testCliDriver_udf_bitwise_and -testCliDriver_udf_bitwise_not -testCliDriver_udf_bitwise_or -testCliDriver_udf_bitwise_xor -testCliDriver_udf_boolean -testCliDriver_udf_case -testCliDriver_udf_case_column_pruning -testCliDriver_udf_case_thrift -testCliDriver_udf_ceil -testCliDriver_udf_ceiling -testCliDriver_udf_coalesce -testCliDriver_udf_compare_java_string -testCliDriver_udf_concat -testCliDriver_udf_concat_insert1 -testCliDriver_udf_concat_insert2 -testCliDriver_udf_concat_ws -testCliDriver_udf_conv -testCliDriver_udf_cos -testCliDriver_udf_count -testCliDriver_udf_date_add -testCliDriver_udf_datediff -testCliDriver_udf_date_sub -testCliDriver_udf_day -testCliDriver_udf_dayofmonth -testCliDriver_udf_degrees -testCliDriver_udf_div -testCliDriver_udf_divide -testCliDriver_udf_double -testCliDriver_udf_E -testCliDriver_udf_elt -testCliDriver_udf_equal -testCliDriver_udf_exp -testCliDriver_udf_explode -testCliDriver_udf_field -testCliDriver_udf_find_in_set -testCliDriver_udf_float -testCliDriver_udf_floor -testCliDriver_udf_format_number -testCliDriver_udf_from_unixtime -testCliDriver_udf_get_json_object -testCliDriver_udf_greaterthan -testCliDriver_udf_greaterthanorequal -testCliDriver_udf_hash -testCliDriver_udf_hex -testCliDriver_udf_hour -testCliDriver_udf_if -testCliDriver_udf_in -testCliDriver_udf_index -testCliDriver_udf_inline -testCliDriver_udf_instr -testCliDriver_udf_int -testCliDriver_udf_isnotnull -testCliDriver_udf_isnull -testCliDriver_udf_isnull_isnotnull -testCliDriver_udf_java_method -testCliDriver_udf_lcase -testCliDriver_udf_length -testCliDriver_udf_lessthan -testCliDriver_udf_lessthanorequal -testCliDriver_udf_like -testCliDriver_udf_ln -testCliDriver_udf_locate -testCliDriver_udf_log -testCliDriver_udf_log10 -testCliDriver_udf_log2 -testCliDriver_udf_logic_java_boolean -testCliDriver_udf_lower -testCliDriver_udf_lpad -testCliDriver_udf_ltrim -testCliDriver_udf_map -testCliDriver_udf_map_keys -testCliDriver_udf_map_values -testCliDriver_udf_max -testCliDriver_udf_min -testCliDriver_udf_minute -testCliDriver_udf_modulo -testCliDriver_udf_month -testCliDriver_udf_named_struct -testCliDriver_udf_negative -testCliDriver_udf_not -testCliDriver_udf_notequal -testCliDriver_udf_notop -testCliDriver_udf_nvl -testCliDriver_udf_or -testCliDriver_udf_parse_url -testCliDriver_udf_percentile -testCliDriver_udf_PI -testCliDriver_udf_pmod -testCliDriver_udf_positive -testCliDriver_udf_pow -testCliDriver_udf_power -testCliDriver_udf_printf -testCliDriver_udf_radians -testCliDriver_udf_rand -testCliDriver_udf_reflect -testCliDriver_udf_reflect2 -testCliDriver_udf_regexp -testCliDriver_udf_regexp_extract -testCliDriver_udf_regexp_replace -testCliDriver_udf_repeat -testCliDriver_udf_reverse -testCliDriver_udf_rlike -testCliDriver_udf_round -testCliDriver_udf_round_2 -testCliDriver_udf_round_3 -testCliDriver_udf_rpad -testCliDriver_udf_rtrim -testCliDriver_udf_second -testCliDriver_udf_sentences -testCliDriver_udf_sign -testCliDriver_udf_sin -testCliDriver_udf_size -testCliDriver_udf_smallint -testCliDriver_udf_sort_array -testCliDriver_udf_space -testCliDriver_udf_split -testCliDriver_udf_sqrt -testCliDriver_udf_std -testCliDriver_udf_stddev -testCliDriver_udf_stddev_pop -testCliDriver_udf_stddev_samp -testCliDriver_udf_string -testCliDriver_udf_struct -testCliDriver_udf_substr -testCliDriver_udf_substring -testCliDriver_udf_subtract -testCliDriver_udf_sum -testCliDriver_udf_tan -testCliDriver_udf_testlength -testCliDriver_udf_testlength2 -testCliDriver_udf_tinyint -testCliDriver_udf_to_date -testCliDriver_udf_to_unix_timestamp -testCliDriver_udf_translate -testCliDriver_udf_trim -testCliDriver_udf_ucase -testCliDriver_udf_unhex -testCliDriver_udf_union -testCliDriver_udf_unix_timestamp -testCliDriver_udf_upper -testCliDriver_udf_variance -testCliDriver_udf_var_pop -testCliDriver_udf_var_samp -testCliDriver_udf_weekofyear -testCliDriver_udf_when -testCliDriver_udf_xpath -testCliDriver_udf_xpath_boolean -testCliDriver_udf_xpath_double -testCliDriver_udf_xpath_float -testCliDriver_udf_xpath_int -testCliDriver_udf_xpath_long -testCliDriver_udf_xpath_short -testCliDriver_udf_xpath_string -testCliDriver_udtf_explode -testCliDriver_udtf_json_tuple -testCliDriver_udtf_parse_url_tuple -testCliDriver_udtf_stack -testCliDriver_union -testCliDriver_union10 -testCliDriver_union11 -testCliDriver_union12 -testCliDriver_union13 -testCliDriver_union14 -testCliDriver_union15 -testCliDriver_union16 -testCliDriver_union18 -testCliDriver_union19 -testCliDriver_union2 -testCliDriver_union20 -testCliDriver_union21 -testCliDriver_union23 -testCliDriver_union25 -testCliDriver_union26 -testCliDriver_union27 -testCliDriver_union28 -testCliDriver_union29 -testCliDriver_union3 -testCliDriver_union30 -testCliDriver_union32 -testCliDriver_union33 -testCliDriver_union34 -testCliDriver_union4 -testCliDriver_union5 -testCliDriver_union6 -testCliDriver_union7 -testCliDriver_union8 -testCliDriver_union9 -testCliDriver_union_lateralview -testCliDriver_union_ppr -testCliDriver_union_remove_6 -testCliDriver_union_script -testCliDriver_union_view -testCliDriver_unset_table_view_property -testCliDriver_updateAccessTime -testCliDriver_view -testCliDriver_view_inputs -testCliDriver_windowing_columnPruning -testCliDriver_windowing_navfn -testCliDriver_windowing_ntile -testCliDriver_windowing_rank -testCliDriver_windowing_udaf -testCliDriver_windowing_windowspec From 93b027fe4bc01692f781a9f95b2f955940ff5408 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Wed, 4 Jun 2014 11:10:12 +0800 Subject: [PATCH 09/21] enable the cli testing --- src/test/scala/shark/CliSuite.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/test/scala/shark/CliSuite.scala b/src/test/scala/shark/CliSuite.scala index 2182e73a..5ead79bc 100644 --- a/src/test/scala/shark/CliSuite.scala +++ b/src/test/scala/shark/CliSuite.scala @@ -21,9 +21,6 @@ import java.io.{BufferedReader, File, InputStreamReader, PrintWriter} import org.scalatest.{BeforeAndAfterAll, FunSuite} -/** - * Test the Shark CLI. -FIX ME class CliSuite extends FunSuite with BeforeAndAfterAll with TestUtils { val WAREHOUSE_PATH = TestUtils.getWarehousePath("cli") @@ -60,4 +57,3 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with TestUtils { } } - */ From d752ed55c7a2e9d4f185095570aa970ac0db46bc Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Wed, 4 Jun 2014 12:05:45 +0800 Subject: [PATCH 10/21] Remove the misktaken commit --- .../tachyon/TachyonOffHeapTableWriter.scala | 58 ------ .../shark/tachyon/TachyonStorageClient.scala | 186 ------------------ .../scala/shark/tachyon/TachyonTableRDD.scala | 104 ---------- 3 files changed, 348 deletions(-) delete mode 100755 src/main/tachyon_enabled/scala/shark/tachyon/TachyonOffHeapTableWriter.scala delete mode 100755 src/main/tachyon_enabled/scala/shark/tachyon/TachyonStorageClient.scala delete mode 100755 src/main/tachyon_enabled/scala/shark/tachyon/TachyonTableRDD.scala diff --git a/src/main/tachyon_enabled/scala/shark/tachyon/TachyonOffHeapTableWriter.scala b/src/main/tachyon_enabled/scala/shark/tachyon/TachyonOffHeapTableWriter.scala deleted file mode 100755 index 522f377c..00000000 --- a/src/main/tachyon_enabled/scala/shark/tachyon/TachyonOffHeapTableWriter.scala +++ /dev/null @@ -1,58 +0,0 @@ -/* -* Copyright (C) 2012 The Regents of The University California. -* All rights reserved. -* -* Licensed 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 shark.tachyon - -import java.nio.ByteBuffer - -import tachyon.client.WriteType - -import shark.LogHelper -import shark.execution.serialization.JavaSerializer -import shark.memstore2.{OffHeapStorageClient, OffHeapTableWriter, TablePartitionStats} - -class TachyonOffHeapTableWriter(@transient path: String, @transient numColumns: Int) - extends OffHeapTableWriter with LogHelper { - - // Re-instantiated upon deserialization, the first time it's referenced. - @transient lazy val tfs = OffHeapStorageClient.client.asInstanceOf[TachyonStorageClient].tfs - - var rawTableId: Int = -1 - - override def createTable() { - val metadata = ByteBuffer.allocate(0) - rawTableId = tfs.createRawTable(path, numColumns, metadata) - } - - override def setStats(indexToStats: collection.Map[Int, TablePartitionStats]) { - val buffer = ByteBuffer.wrap(JavaSerializer.serialize(indexToStats)) - tfs.updateRawTableMetadata(rawTableId, buffer) - } - - // rawTable is a lazy val so it gets created the first time it is referenced. - // This is only used on worker nodes. - @transient lazy val rawTable = tfs.getRawTable(rawTableId) - - override def writeColumnPartition(column: Int, part: Int, data: ByteBuffer) { - val rawColumn = rawTable.getRawColumn(column) - rawColumn.createPartition(part) - val file = rawColumn.getPartition(part) - val outStream = file.getOutStream(WriteType.CACHE_THROUGH) - outStream.write(data.array(), 0, data.limit()) - outStream.close() - } -} diff --git a/src/main/tachyon_enabled/scala/shark/tachyon/TachyonStorageClient.scala b/src/main/tachyon_enabled/scala/shark/tachyon/TachyonStorageClient.scala deleted file mode 100755 index a494578d..00000000 --- a/src/main/tachyon_enabled/scala/shark/tachyon/TachyonStorageClient.scala +++ /dev/null @@ -1,186 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark.tachyon - -import java.util.{BitSet => JBitSet} -import java.util.concurrent.{ConcurrentHashMap => ConcurrentJavaHashMap} - -import scala.collection.JavaConversions._ - -import org.apache.spark.rdd.{RDD, UnionRDD} -import tachyon.client.TachyonFS - -import shark.{LogHelper, SharkEnv} -import shark.execution.TableReader.PruningFunctionType -import shark.execution.serialization.JavaSerializer -import shark.memstore2.{OffHeapStorageClient, OffHeapStorageClientFactory, TablePartitionStats} - -class TachyonStorageClientFactory extends OffHeapStorageClientFactory { - def createClient() = { - new TachyonStorageClient( - System.getenv("TACHYON_MASTER"), System.getenv("TACHYON_WAREHOUSE_PATH")) - } -} - -/** - * Enables caching of Shark tables in native column-oriented format into Tachyon. - * - * The directory structure for a given table in Tachyon looks like: - * Data: warehouse/database.table/_defaultkey/insert_#/col_#/part_# - * Metadata: warehouse/database.table/_defaultkey/insert_#/.meta - * where: - * - insert_# is used to allow inserting data multiple times. Files cannot be appended to in - * Tachyon, so we instead create a whole new directory and union them upon read. - * - col_# is the folder for the particular column - * - part_# is the Spark partition for the column - * - * Note that "_defaultkey" is the name of the Hive Partition for a non-partitioned table. If the - * table is partitioned, it will be replaced by each "hivePartitionKey". - */ -class TachyonStorageClient(val master: String, val warehousePath: String) - extends OffHeapStorageClient with LogHelper { - - /** We create a new directory with a new RawTable for each independent insert. */ - private val INSERT_FILE_PREFIX = "insert_" - - /** Non-partitioned tables use a default partition name for consistency. */ - private val DEFAULT_PARTITION = "_defaultkey" - - private val _fileNameMappings = new ConcurrentJavaHashMap[String, Int]() - - if (master == null || master == "") { - throw new TachyonException("TACHYON_MASTER is not set, cannot create TachyonStorageClient.") - } - if (warehousePath == null) { - throw new TachyonException("TACHYON_WAREHOUSE is not set, cannot create TachyonStorageClient.") - } - - val tfs = TachyonFS.get(master) - - private def getUniqueFilePath(parentDirectory: String): String = { - val parentDirectoryLower = parentDirectory.toLowerCase - val currentInsertNum = if (_fileNameMappings.containsKey(parentDirectoryLower)) { - _fileNameMappings.get(parentDirectoryLower) - } else { - 0 - } - var nextInsertNum = currentInsertNum + 1 - val filePath = parentDirectoryLower + "/" + INSERT_FILE_PREFIX - // Make sure there aren't file conflicts. This could occur if the directory was created in a - // previous Shark session. - while (tfs.exist(filePath + nextInsertNum)) { - nextInsertNum = nextInsertNum + 1 - } - _fileNameMappings.put(parentDirectoryLower, nextInsertNum) - filePath + nextInsertNum - } - - private def getTablePath(tableKey: String): String = { - warehousePath + "/" + tableKey - } - - private def getPartitionPath(tableKey: String, hivePartitionKey: String): String = { - getTablePath(tableKey) + "/" + hivePartitionKey - } - - override def tableExists(tableKey: String): Boolean = { - tfs.exist(getTablePath(tableKey)) - } - - override def tablePartitionExists(tableKey: String, hivePartitionKey: Option[String]): Boolean = { - tfs.exist(getPartitionPath(tableKey, hivePartitionKey.getOrElse(DEFAULT_PARTITION))) - } - - override def dropTable(tableKey: String): Boolean = { - tfs.delete(getTablePath(tableKey), true /* recursively */) - } - - override def dropTablePartition(tableKey: String, hivePartitionKey: Option[String]): Boolean = { - tfs.delete(getPartitionPath(tableKey, hivePartitionKey.getOrElse(DEFAULT_PARTITION)), - true /* recursively */) - } - - /** - * Reads a particular Hive partition (or whole table if non-partitioned) into a single RDD. - * Since each insert is written to its own directory, we need to reconstruct this RDD by reading - * across all of these insert directories. We thus column- and row-prune our data before - * producing the union to avoid data transfer. - * - * @param columnsUsed Indicates which columns are needed, to avoid loading extra data. - * @param pruningFn Used for pruning rows. - */ - override def readTablePartition( - tableKey: String, - hivePartitionKey: Option[String], - columnsUsed: JBitSet, - pruningFn: PruningFunctionType - ): RDD[_] = { - - try { - if (!tablePartitionExists(tableKey, hivePartitionKey)) { - throw new TachyonException("Table " + tableKey + " does not exist in Tachyon") - } - - // Create a TachyonTableRDD for each raw tableRDDsAndStats file in the directory. - val tableDirectory = getPartitionPath(tableKey, hivePartitionKey.getOrElse(DEFAULT_PARTITION)) - val files = tfs.ls(tableDirectory, false /* recursive */) - // The first path is just "{tableDirectory}/", so ignore it. - val rawTableFiles = files.subList(1, files.size) - val prunedRDDs = rawTableFiles.map { filePath => - val serializedMetadata = tfs.getRawTable(tfs.getFileId(filePath)).getMetadata - val indexToStats = JavaSerializer.deserialize[collection.Map[Int, TablePartitionStats]]( - serializedMetadata.array()) - pruningFn(new TachyonTableRDD(filePath, columnsUsed, SharkEnv.sc), indexToStats) - } - new UnionRDD(SharkEnv.sc, prunedRDDs.toSeq.asInstanceOf[Seq[RDD[Any]]]) - } catch { - case e: Exception => - logError("Exception while reading table partition", e) - throw e - } - } - - override def createTablePartitionWriter( - tableKey: String, - hivePartitionKey: Option[String], - numColumns: Int): TachyonOffHeapTableWriter = { - if (!tfs.exist(warehousePath)) { - tfs.mkdir(warehousePath) - } - val parentDirectory = getPartitionPath(tableKey, hivePartitionKey.getOrElse(DEFAULT_PARTITION)) - val filePath = getUniqueFilePath(parentDirectory) - new TachyonOffHeapTableWriter(filePath, numColumns) - } - - override def createTablePartition( - tableKey: String, - hivePartitionKeyOpt: Option[String]): Boolean = { - hivePartitionKeyOpt match { - case Some(key) => tfs.mkdir(getPartitionPath(tableKey, key)) - case None => tfs.mkdir(getPartitionPath(tableKey, DEFAULT_PARTITION)) - } - } - - override def renameTable( - oldTableKey: String, - newTableKey: String): Boolean = { - val oldPath = getTablePath(oldTableKey) - val newPath = getTablePath(newTableKey) - tfs.rename(oldPath, newPath) - } -} diff --git a/src/main/tachyon_enabled/scala/shark/tachyon/TachyonTableRDD.scala b/src/main/tachyon_enabled/scala/shark/tachyon/TachyonTableRDD.scala deleted file mode 100755 index 59ca6b0c..00000000 --- a/src/main/tachyon_enabled/scala/shark/tachyon/TachyonTableRDD.scala +++ /dev/null @@ -1,104 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark.tachyon - -import java.nio.{ByteBuffer, ByteOrder} -import java.util.{BitSet => JBitSet} - -import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.{Partition, SparkContext, TaskContext} -import shark.memstore2._ -import tachyon.client.{ReadType, TachyonByteBuffer} -import tachyon.client.table.RawTable -import org.apache.spark.rdd.RDD -import shark.LogHelper - -private class TachyonTablePartition(rddId: Int, idx: Int, val locations: Seq[String]) - extends Partition { - - override def hashCode(): Int = (41 * (41 + rddId) + idx).toInt - - override val index: Int = idx -} - -/** - * An RDD that reads a Tachyon Table. - */ -class TachyonTableRDD(path: String, columnsUsed: JBitSet, @transient sc: SparkContext) - extends RDD[TablePartition](sc, Nil) with LogHelper { - - @transient lazy val tfs = OffHeapStorageClient.client.asInstanceOf[TachyonStorageClient].tfs - - override def getPartitions: Array[Partition] = { - val rawTable: RawTable = tfs.getRawTable(path) - // Use the first column to get preferred locations for all partitions. - val rawColumn = rawTable.getRawColumn(0) - val numPartitions: Int = rawColumn.partitions() - Array.tabulate[Partition](numPartitions) { part => - val locations = rawColumn.getPartition(part).getLocationHosts().asScala - new TachyonTablePartition(id, part, locations) : Partition - } - } - - override def compute(theSplit: Partition, context: TaskContext): Iterator[TablePartition] = { - val rawTable: RawTable = tfs.getRawTable(path) - val activeBuffers = new ArrayBuffer[TachyonByteBuffer]() - val buffers = Array.tabulate[ByteBuffer](rawTable.getColumns()) { columnIndex => - if (columnIndex != 0 && columnsUsed != null && !columnsUsed.get(columnIndex - 1)) { - null - } else { - val fp = rawTable.getRawColumn(columnIndex).getPartition(theSplit.index, true) - // Try to read data from Tachyon's memory, either local or remote. - var buf = fp.readByteBuffer() - if (buf == null && fp.recache()) { - // The data is not in Tachyon's memory yet, recache succeed. - buf = fp.readByteBuffer() - } - if (buf == null) { - logWarning("Table " + path + " column " + columnIndex + " partition " + theSplit.index - + " is not in Tachyon's memory. Streaming it in.") - var data = ByteBuffer.allocate(fp.length().toInt) - val is = fp.getInStream(ReadType.CACHE) - is.read(data.array) - is.close() - data.limit(fp.length().toInt) - data - } else { - activeBuffers += buf - buf.DATA - } - } - } - - // Register an on-task-completion callback to close the input stream. - context.addOnCompleteCallback(() => activeBuffers.foreach(_.close())) - - Iterator(new TablePartition(buffers.map(buffer => - if (buffer == null) null else buffer.order(ByteOrder.nativeOrder())))) - } - - override def getPreferredLocations(split: Partition): Seq[String] = { - split.asInstanceOf[TachyonTablePartition].locations - } - - // override def checkpoint() { - // // Do nothing. Tachyon RDD should not be checkpointed. - // } -} From 6e7b4d2af7cd2e2b91389fa91787c38c738d256e Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Wed, 4 Jun 2014 12:24:53 +0800 Subject: [PATCH 11/21] Add some document --- src/main/scala/shark/CatalystContext.scala | 11 +++++++++-- src/main/scala/shark/CatalystDriver.scala | 8 +++++++- src/main/scala/shark/CatalystEnv.scala | 1 + src/main/scala/shark/SharkCliDriver.scala | 2 ++ 4 files changed, 19 insertions(+), 3 deletions(-) diff --git a/src/main/scala/shark/CatalystContext.scala b/src/main/scala/shark/CatalystContext.scala index 760dee33..230cd969 100644 --- a/src/main/scala/shark/CatalystContext.scala +++ b/src/main/scala/shark/CatalystContext.scala @@ -45,7 +45,11 @@ case class CatalystContext(sc: SparkContext) extends HiveContext(sc) with LogHel class HiveQLQueryExecution(hql: String) extends QueryExecution { override def logical: LogicalPlan = HiveQl.parseSql(hql) override def toString = hql + "\n" + super.toString - + + /** + * Query Result (errcode, result, exception if any) + * If error code equals 0 means got the result, otherwise failed due to some reason / exception + */ def result(): (Int, Seq[String], Throwable) = analyzed match { case NativeCommand(cmd) => runOnHive(cmd) case ExplainCommand(plan) => @@ -64,7 +68,10 @@ case class CatalystContext(sc: SparkContext) extends HiveContext(sc) with LogHel } } } - + + /** + * Get the result set table schema + */ def getResultSetSchema: TableSchema = { logger.warn(s"Result Schema: ${analyzed.output}") if (analyzed.output.size == 0) { diff --git a/src/main/scala/shark/CatalystDriver.scala b/src/main/scala/shark/CatalystDriver.scala index b21428c3..4498a5ab 100644 --- a/src/main/scala/shark/CatalystDriver.scala +++ b/src/main/scala/shark/CatalystDriver.scala @@ -36,7 +36,13 @@ class CatalystDriver(hconf: HiveConf) extends Driver { 0 } - + + /** + * Get the result schema, currently CatalystDriver doesn't support it yet. + * TODO: the TableSchema (org.apache.hive.service.cli.TableSchema) is returned by Catalyst, + * however, the Driver requires the Schema (org.apache.hadoop.hive.metastore.api.Schema) + * Need to figure out how to convert the previous to later. + */ override def getSchema(): Schema = throw new UnsupportedOperationException("for getSchema") def getTableSchema = tschema diff --git a/src/main/scala/shark/CatalystEnv.scala b/src/main/scala/shark/CatalystEnv.scala index ac17f760..eb94de7c 100755 --- a/src/main/scala/shark/CatalystEnv.scala +++ b/src/main/scala/shark/CatalystEnv.scala @@ -26,6 +26,7 @@ import org.apache.spark.SparkContext import org.apache.spark.sql.hive.CatalystContext /** A singleton object for the master program. The slaves should not access this. */ +// TODO add tachyon / memory store based (Copied from SharkEnv.scala) object CatalystEnv extends LogHelper { def init(): CatalystContext = { diff --git a/src/main/scala/shark/SharkCliDriver.scala b/src/main/scala/shark/SharkCliDriver.scala index e118cfdf..c385f890 100755 --- a/src/main/scala/shark/SharkCliDriver.scala +++ b/src/main/scala/shark/SharkCliDriver.scala @@ -273,6 +273,7 @@ class SharkCliDriver(reloadRdds: Boolean = true) extends CliDriver with LogHelpe // because the Hive unit tests do not go through the main() code path. if (!ss.isRemoteMode()) { CatalystEnv.init() +// TODO reload the rdds // if (reloadRdds) { // console.printInfo( // "Reloading cached RDDs from previous Shark sessions... (use %s flag to skip reloading)" @@ -344,6 +345,7 @@ class SharkCliDriver(reloadRdds: Boolean = true) extends CliDriver with LogHelpe if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_CLI_PRINT_HEADER)) { // Print the column names. + // TODO currently CatalystDriver returns the TableSchema instead of the Schema if(qp.isInstanceOf[CatalystDriver]) { val fieldDescs = qp.asInstanceOf[CatalystDriver].getTableSchema.getColumnDescriptors() if (fieldDescs != null) { From 3050f80b8c046e93d70b576faa2b773a44c22e7f Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Wed, 4 Jun 2014 15:04:09 +0800 Subject: [PATCH 12/21] Add CacheRdd reload support --- src/main/scala/shark/CatalystContext.scala | 20 ++++-- src/main/scala/shark/CatalystDriver.scala | 5 +- src/main/scala/shark/CatalystEnv.scala | 23 ++++++- src/main/scala/shark/SharkCliDriver.scala | 17 +++-- .../scala/shark/memstore2/CacheType.scala | 64 +++++++++++++++++ .../shark/memstore2/SharkTblProperties.scala | 68 ++++++++++++++++++ .../scala/shark/memstore2/TableRecovery.scala | 69 +++++++++++++++++++ .../scala/shark/util/QueryRewriteUtils.scala | 50 ++++++++++++++ 8 files changed, 297 insertions(+), 19 deletions(-) create mode 100644 src/main/scala/shark/memstore2/CacheType.scala create mode 100644 src/main/scala/shark/memstore2/SharkTblProperties.scala create mode 100644 src/main/scala/shark/memstore2/TableRecovery.scala create mode 100644 src/main/scala/shark/util/QueryRewriteUtils.scala diff --git a/src/main/scala/shark/CatalystContext.scala b/src/main/scala/shark/CatalystContext.scala index 230cd969..56663a79 100644 --- a/src/main/scala/shark/CatalystContext.scala +++ b/src/main/scala/shark/CatalystContext.scala @@ -102,15 +102,21 @@ case class CatalystContext(sc: SparkContext) extends HiveContext(sc) with LogHel // Throw an exception if there is an error in query processing. if (response.getResponseCode != 0) { driver.destroy() - throw new QueryExecutionException(response.getErrorMessage) + (response.getResponseCode, Seq[String](response.getErrorMessage()), new Exception(cmd)) + } else { + driver.setMaxRows(maxRows) + driver.getResults(results) + driver.destroy() + (0, results, null) } - driver.setMaxRows(maxRows) - driver.getResults(results) - driver.destroy() - (0, results, null) case _ => SessionState.get().out.println(tokens(0) + " " + cmd_1) - (proc.run(cmd_1).getResponseCode, Seq[String](), null) + val res = proc.run(cmd_1) + if(res.getResponseCode == 0) { + (0, Seq[String](), null) + } else { + (res.getResponseCode, Seq[String](res.getErrorMessage()), new Exception(cmd_1)) + } } } catch { case e: Throwable => @@ -124,7 +130,7 @@ case class CatalystContext(sc: SparkContext) extends HiveContext(sc) with LogHel |END HIVE FAILURE OUTPUT |====================== """.stripMargin) - (-2, Seq[String](), null) + (-2, Seq[String](), e) } } } diff --git a/src/main/scala/shark/CatalystDriver.scala b/src/main/scala/shark/CatalystDriver.scala index 4498a5ab..b14582f1 100644 --- a/src/main/scala/shark/CatalystDriver.scala +++ b/src/main/scala/shark/CatalystDriver.scala @@ -10,7 +10,7 @@ import org.apache.spark.sql.hive.CatalystContext import scala.collection.JavaConversions._ import org.apache.commons.lang.exception.ExceptionUtils -class CatalystDriver(hconf: HiveConf) extends Driver { +class CatalystDriver(hconf: HiveConf) extends Driver with LogHelper { private val context: CatalystContext = CatalystEnv.cc private var tschema: TableSchema = _ private var result: (Int, Seq[String], Throwable) = _ @@ -24,7 +24,8 @@ class CatalystDriver(hconf: HiveConf) extends Driver { tschema = execution.getResultSetSchema if(result._1 != 0) { - new CommandProcessorResponse(result._1, ExceptionUtils.getStackTrace(result._3), null) + logError(s"Failed in [$command]", result._3) + new CommandProcessorResponse(result._1, ExceptionUtils.getFullStackTrace(result._3), null) } else { new CommandProcessorResponse(result._1) } diff --git a/src/main/scala/shark/CatalystEnv.scala b/src/main/scala/shark/CatalystEnv.scala index eb94de7c..6a060272 100755 --- a/src/main/scala/shark/CatalystEnv.scala +++ b/src/main/scala/shark/CatalystEnv.scala @@ -24,6 +24,7 @@ import org.apache.spark.SparkConf import org.apache.spark.scheduler.StatsReportListener import org.apache.spark.SparkContext import org.apache.spark.sql.hive.CatalystContext +import org.apache.spark.scheduler.SplitInfo /** A singleton object for the master program. The slaves should not access this. */ // TODO add tachyon / memory store based (Copied from SharkEnv.scala) @@ -94,17 +95,33 @@ object CatalystEnv extends LogHelper { if (sc != null) { sc.stop() } - + sc = new SparkContext( - if (master == null) "local" else master, + createSparkConf(if (master == null) "local" else master, jobName, System.getenv("SPARK_HOME"), Nil, - executorEnvVars) + executorEnvVars), Map[String, Set[SplitInfo]]()) sc } + private def createSparkConf( + master: String, + jobName: String, + sparkHome: String, + jars: Seq[String], + environment: HashMap[String, String]): SparkConf = { + val newConf = new SparkConf() + .setMaster(master) + .setAppName(jobName) + .setJars(jars) + .setExecutorEnv(environment.toSeq) + Option(sparkHome).foreach(newConf.setSparkHome(_)) + + newConf + } + logDebug("Initializing SharkEnv") val executorEnvVars = new HashMap[String, String] diff --git a/src/main/scala/shark/SharkCliDriver.scala b/src/main/scala/shark/SharkCliDriver.scala index c385f890..7e8139b1 100755 --- a/src/main/scala/shark/SharkCliDriver.scala +++ b/src/main/scala/shark/SharkCliDriver.scala @@ -37,6 +37,7 @@ import org.apache.hadoop.hive.cli.{CliDriver, CliSessionState, OptionsProcessor} import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils, LogUtils} import org.apache.hadoop.hive.common.LogUtils.LogInitializationException import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.hive.ql.processors.{CommandProcessor, CommandProcessorFactory} @@ -45,6 +46,8 @@ import org.apache.hadoop.hive.shims.ShimLoader import org.apache.hadoop.io.IOUtils import org.apache.thrift.transport.TSocket +import shark.memstore2.TableRecovery + object SharkCliDriver { val SKIP_RDD_RELOAD_FLAG = "-skipRddReload" @@ -96,6 +99,7 @@ object SharkCliDriver { } val ss = new CliSessionState(new HiveConf(classOf[SessionState])) + ss.in = System.in try { ss.out = new PrintStream(System.out, true, "UTF-8") @@ -273,13 +277,12 @@ class SharkCliDriver(reloadRdds: Boolean = true) extends CliDriver with LogHelpe // because the Hive unit tests do not go through the main() code path. if (!ss.isRemoteMode()) { CatalystEnv.init() -// TODO reload the rdds -// if (reloadRdds) { -// console.printInfo( -// "Reloading cached RDDs from previous Shark sessions... (use %s flag to skip reloading)" -// .format(SharkCliDriver.SKIP_RDD_RELOAD_FLAG)) -// TableRecovery.reloadRdds(processCmd(_), Some(console)) -// } + if (reloadRdds) { + console.printInfo( + "Reloading cached RDDs from previous Shark sessions... (use %s flag to skip reloading)" + .format(SharkCliDriver.SKIP_RDD_RELOAD_FLAG)) + TableRecovery.reloadRdds(processCmd(_), Some(console), ss) + } } def this() = this(false) diff --git a/src/main/scala/shark/memstore2/CacheType.scala b/src/main/scala/shark/memstore2/CacheType.scala new file mode 100644 index 00000000..2e9773ac --- /dev/null +++ b/src/main/scala/shark/memstore2/CacheType.scala @@ -0,0 +1,64 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2 + +import shark.LogHelper + + +/* + * Enumerations and static helper functions for caches supported by Shark. + */ +object CacheType extends Enumeration with LogHelper { + + /* + * The CacheTypes: + * - MEMORY: Stored in memory and on disk (i.e., cache is write-through). Persistent across Shark + * sessions. By default, all such tables are reloaded into memory on restart. + * - MEMORY_ONLY: Stored only in memory and dropped at the end of each Shark session. + * - OFFHEAP: Stored in an off-heap data storage format, specified by the System property + * 'shark.offheap.clientFactory'. Defaults to TachyonStorageClientFactory. + * - NONE: Stored on disk (e.g., HDFS) and managed by Hive. + */ + type CacheType = Value + val MEMORY, MEMORY_ONLY, OFFHEAP, NONE = Value + + def shouldCache(c: CacheType): Boolean = (c != NONE) + + /** Get the cache type object from a string representation. */ + def fromString(name: String): CacheType = Option(name).map(_.toUpperCase) match { + case None | Some("") | Some("FALSE") => NONE + case Some("TRUE") => MEMORY + case Some("HEAP") => + logWarning("The 'HEAP' cache type name is deprecated. Use 'MEMORY' instead.") + MEMORY + case Some("TACHYON") => + logWarning("The 'TACHYON' cache type name is deprecated. Use 'OFFHEAP' instead.") + OFFHEAP + case _ => { + try { + // Try to use Scala's Enumeration::withName() to interpret 'name'. + withName(name.toUpperCase) + } catch { + case e: java.util.NoSuchElementException => throw new InvalidCacheTypeException(name) + } + } + } + + class InvalidCacheTypeException(name: String) + extends Exception("Invalid string representation of cache type: '%s'".format(name)) +} diff --git a/src/main/scala/shark/memstore2/SharkTblProperties.scala b/src/main/scala/shark/memstore2/SharkTblProperties.scala new file mode 100644 index 00000000..befc91d1 --- /dev/null +++ b/src/main/scala/shark/memstore2/SharkTblProperties.scala @@ -0,0 +1,68 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2 + +import java.util.{Map => JavaMap} + + +/** + * Collection of static fields and helpers for table properties (i.e., from A + * CREATE TABLE TBLPROPERTIES( ... ) used by Shark. + */ +object SharkTblProperties { + + case class TableProperty(varname: String, defaultVal: String) + + // Class name of the default cache policy used to manage partition evictions for cached, + // Hive-partitioned tables. + val CACHE_POLICY = new TableProperty("shark.cache.policy", "shark.memstore2.CacheAllPolicy") + + // Maximum size - in terms of the number of objects - of the cache specified by the + // "shark.cache.partition.cachePolicy" property above. + val MAX_PARTITION_CACHE_SIZE = new TableProperty("shark.cache.policy.maxSize", "10") + + // Default value for the "shark.cache" table property + val CACHE_FLAG = new TableProperty("shark.cache", "true") + + def getOrSetDefault(tblProps: JavaMap[String, String], variable: TableProperty): String = { + if (!tblProps.containsKey(variable.varname)) { + tblProps.put(variable.varname, variable.defaultVal) + } + tblProps.get(variable.varname) + } + + /** + * Returns value for the `variable` table property. If a value isn't present in `tblProps`, then + * the default for `variable` will be returned. + */ + def initializeWithDefaults( + tblProps: JavaMap[String, String], + isPartitioned: Boolean = false): JavaMap[String, String] = { + tblProps.put(CACHE_FLAG.varname, CACHE_FLAG.defaultVal) + if (isPartitioned) { + tblProps.put(CACHE_POLICY.varname, CACHE_POLICY.defaultVal) + } + tblProps + } + + def removeSharkProperties(tblProps: JavaMap[String, String]) { + tblProps.remove(CACHE_FLAG.varname) + tblProps.remove(CACHE_POLICY.varname) + tblProps.remove(MAX_PARTITION_CACHE_SIZE.varname) + } +} diff --git a/src/main/scala/shark/memstore2/TableRecovery.scala b/src/main/scala/shark/memstore2/TableRecovery.scala new file mode 100644 index 00000000..faaa4fbf --- /dev/null +++ b/src/main/scala/shark/memstore2/TableRecovery.scala @@ -0,0 +1,69 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.memstore2 + +import java.util.{HashMap => JavaHashMap} + +import scala.collection.JavaConversions.asScalaBuffer + +import org.apache.hadoop.hive.ql.metadata.Hive +import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hadoop.hive.cli.CliSessionState + +import shark.{LogHelper, CatalystEnv} +import shark.util.QueryRewriteUtils + +/** + * Singleton used to reload RDDs upon server restarts. + */ +object TableRecovery extends LogHelper { + + val db = Hive.get() + + /** + * Loads any cached tables with MEMORY as its `shark.cache` property. + * @param cmdRunner The runner that is responsible for taking a cached table query and + * a) Creating the table metadata in Hive Meta Store + * b) Loading the table as an RDD in memory + * @see SharkServer for an example usage. + * @param console Optional SessionState.LogHelper used, if present, to log information about + the tables that get reloaded. + */ + def reloadRdds(cmdRunner: String => Unit, console: Option[SessionState.LogHelper] = None, + ss: CliSessionState) { + // Filter for tables that should be reloaded into the cache. + val currentDbName = ss.getCurrentDatabase() + for (databaseName <- db.getAllDatabases(); tableName <- db.getAllTables(databaseName)) { + val hiveTable = db.getTable(databaseName, tableName) + val tblProps = hiveTable.getParameters + val cacheMode = CacheType.fromString(tblProps.get(SharkTblProperties.CACHE_FLAG.varname)) + if (cacheMode == CacheType.MEMORY) { + val logMessage = "Reloading %s.%s into memory.".format(databaseName, tableName) + if (console.isDefined) { + console.get.printInfo(logMessage) + } else { + logInfo(logMessage) + } + val cmd = QueryRewriteUtils.cacheToAlterTable("CACHE %s".format(tableName)) + cmdRunner(s"use $databaseName") + cmdRunner(cmd) + } + } + ss.setCurrentDatabase(currentDbName) + } +} diff --git a/src/main/scala/shark/util/QueryRewriteUtils.scala b/src/main/scala/shark/util/QueryRewriteUtils.scala new file mode 100644 index 00000000..de43ca01 --- /dev/null +++ b/src/main/scala/shark/util/QueryRewriteUtils.scala @@ -0,0 +1,50 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark.util + +import org.apache.hadoop.hive.ql.parse.SemanticException + +object QueryRewriteUtils { + + def cacheToAlterTable(cmd: String): String = { + val CACHE_TABLE_DEFAULT = "(?i)CACHE ([^ ]+)".r + val CACHE_TABLE_IN = "(?i)CACHE ([^ ]+) IN ([^ ]+)".r + + cmd match { + case CACHE_TABLE_DEFAULT(tableName) => + s"ALTER TABLE $tableName SET TBLPROPERTIES ('shark.cache' = 'memory')" + case CACHE_TABLE_IN(tableName, cacheType) => + s"ALTER TABLE $tableName SET TBLPROPERTIES ('shark.cache' = '$cacheType')" + case _ => + throw new SemanticException( + s"CACHE accepts a single table name: 'CACHE [IN ]'" + + s" (received command: '$cmd')") + } + } + + def uncacheToAlterTable(cmd: String): String = { + val cmdSplit = cmd.split(' ') + if (cmdSplit.size == 2) { + val tableName = cmdSplit(1) + "ALTER TABLE %s SET TBLPROPERTIES ('shark.cache' = 'false')".format(tableName) + } else { + throw new SemanticException( + s"UNCACHE accepts a single table name: 'UNCACHE
' (received command: '$cmd')") + } + } +} From 3e652fe222b9a72eece91cf0f6e86163fc1f622b Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Wed, 4 Jun 2014 15:18:16 +0800 Subject: [PATCH 13/21] Update ReadMe for supporting the cached reload --- README.md | 1 - 1 file changed, 1 deletion(-) diff --git a/README.md b/README.md index 35bb025c..960cbb4d 100755 --- a/README.md +++ b/README.md @@ -72,7 +72,6 @@ hive>show tables; But there is a bug, which require show tables before doing anything else. ## Known Missing Features -* Restoring cached tables upon restart * Invalidation of cached tables when data is INSERTed * Off-heap storage using Tachyon * TGFs From ca6255f71c253c3fa3391ce2a87fe682ea12450e Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Wed, 4 Jun 2014 15:29:49 +0800 Subject: [PATCH 14/21] Output Error Message for HQL --- src/main/scala/shark/CatalystDriver.scala | 24 +++++++++++++++-------- 1 file changed, 16 insertions(+), 8 deletions(-) diff --git a/src/main/scala/shark/CatalystDriver.scala b/src/main/scala/shark/CatalystDriver.scala index b14582f1..327d0f84 100644 --- a/src/main/scala/shark/CatalystDriver.scala +++ b/src/main/scala/shark/CatalystDriver.scala @@ -20,14 +20,22 @@ class CatalystDriver(hconf: HiveConf) extends Driver with LogHelper { override def run(command: String): CommandProcessorResponse = { val execution = new context.HiveQLQueryExecution(command) - result = execution.result - tschema = execution.getResultSetSchema - - if(result._1 != 0) { - logError(s"Failed in [$command]", result._3) - new CommandProcessorResponse(result._1, ExceptionUtils.getFullStackTrace(result._3), null) - } else { - new CommandProcessorResponse(result._1) + + // TODO unify the error code + try { + result = execution.result + tschema = execution.getResultSetSchema + + if(result._1 != 0) { + logError(s"Failed in [$command]", result._3) + new CommandProcessorResponse(result._1, ExceptionUtils.getFullStackTrace(result._3), null) + } else { + new CommandProcessorResponse(result._1) + } + } catch { + case t: Throwable => + logError(s"Failed in [$command]", t) + new CommandProcessorResponse(-3, ExceptionUtils.getFullStackTrace(t), null) } } From b5c031bc5372b08f61cb62e3517219354dc3f5fb Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 5 Jun 2014 08:54:20 +0800 Subject: [PATCH 15/21] solve the netty / servlet-api jar conflict --- project/SharkBuild.scala | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index 391c80ff..819340d6 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -32,7 +32,7 @@ object SharkBuild extends Build { val SHARK_ORGANIZATION = "edu.berkeley.cs.shark" - val SPARK_VERSION = "1.0.0-SNAPSHOT" + val SPARK_VERSION = "1.1.0-SNAPSHOT" val SCALA_VERSION = "2.10.4" @@ -80,7 +80,8 @@ object SharkBuild extends Build { /** Extra artifacts not included in Spark SQL's Hive support. */ val hiveArtifacts = Seq("hive-cli", "hive-jdbc", "hive-exec", "hive-service") val hiveDependencies = hiveArtifacts.map ( artifactId => - "org.spark-project.hive" % artifactId % "0.12.0" + "org.spark-project.hive" % artifactId % "0.12.0" excludeAll( + excludeGuava, excludeLog4j, excludeServlet, excludeAsm, excludeNetty, excludeXerces) ) val yarnDependency = (if (YARN_ENABLED) { @@ -101,12 +102,9 @@ object SharkBuild extends Build { libraryDependencies ++= hiveDependencies ++ yarnDependency, libraryDependencies ++= Seq( - "io.netty" % "netty-all" % "4.0.17.Final", - "org.eclipse.jetty" % "jetty-server" % JETTY_VERSION, - "org.eclipse.jetty" % "jetty-util" % JETTY_VERSION, - "javax.servlet" % "javax.servlet-api" % "3.0.1", "org.apache.spark" %% "spark-hive" % SPARK_VERSION, "org.apache.spark" %% "spark-repl" % SPARK_VERSION, + "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm) force(), "com.typesafe" %% "scalalogging-slf4j" % "1.0.1", "org.scalatest" %% "scalatest" % "1.9.1" % "test" ), From da57ff6a67bfecde3e395f736eafdf87f6dc0fed Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 5 Jun 2014 22:04:31 +0800 Subject: [PATCH 16/21] Jar conflict & Work around for CliSessionState modified by HiveContext --- project/SharkBuild.scala | 12 ++++--- src/main/scala/shark/CatalystContext.scala | 5 ++- src/main/scala/shark/CatalystDriver.scala | 31 ++++++++++++++++--- src/main/scala/shark/CatalystEnv.scala | 2 +- src/main/scala/shark/SharkCliDriver.scala | 20 ++++++++---- src/main/scala/shark/SharkServer2.scala | 27 +++++++++------- .../scala/shark/server/SharkCLIService.scala | 6 ++-- 7 files changed, 72 insertions(+), 31 deletions(-) diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index 819340d6..bd2d72dc 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -72,16 +72,16 @@ object SharkBuild extends Build { // Exclusion rules for Hive artifacts val excludeGuava = ExclusionRule(organization = "com.google.guava") val excludeLog4j = ExclusionRule(organization = "log4j") - val excludeServlet = ExclusionRule(organization = "org.mortbay.jetty") + val excludeJetty = ExclusionRule(organization = "org.mortbay.jetty") val excludeXerces = ExclusionRule(organization = "xerces") val excludeHive = ExclusionRule(organization = "org.apache.hive") - /** Extra artifacts not included in Spark SQL's Hive support. */ val hiveArtifacts = Seq("hive-cli", "hive-jdbc", "hive-exec", "hive-service") + val hiveDependencies = hiveArtifacts.map ( artifactId => "org.spark-project.hive" % artifactId % "0.12.0" excludeAll( - excludeGuava, excludeLog4j, excludeServlet, excludeAsm, excludeNetty, excludeXerces) + excludeGuava, excludeLog4j, excludeAsm, excludeJetty, excludeNetty, excludeXerces) ) val yarnDependency = (if (YARN_ENABLED) { @@ -102,9 +102,13 @@ object SharkBuild extends Build { libraryDependencies ++= hiveDependencies ++ yarnDependency, libraryDependencies ++= Seq( - "org.apache.spark" %% "spark-hive" % SPARK_VERSION, + "org.apache.spark" %% "spark-hive" % SPARK_VERSION excludeAll(excludeHive, excludeJetty) force(), "org.apache.spark" %% "spark-repl" % SPARK_VERSION, "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm) force(), + "org.mortbay.jetty" % "jetty" % "6.1.26" exclude ("org.mortbay.jetty", "servlet-api") force(), + "org.eclipse.jetty.orbit" % "javax.servlet" % "3.0.0.v201112011016" artifacts ( Artifact("javax.servlet", "jar", "jar") ), + "com.google.guava" % "guava" % "14.0.1", + "commons-io" % "commons-io" % "2.1", "com.typesafe" %% "scalalogging-slf4j" % "1.0.1", "org.scalatest" %% "scalatest" % "1.9.1" % "test" ), diff --git a/src/main/scala/shark/CatalystContext.scala b/src/main/scala/shark/CatalystContext.scala index 56663a79..0852ca46 100644 --- a/src/main/scala/shark/CatalystContext.scala +++ b/src/main/scala/shark/CatalystContext.scala @@ -33,7 +33,6 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse import org.apache.hadoop.hive.ql.Driver import org.apache.spark.SparkContext -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.plans.logical.NativeCommand import org.apache.spark.sql.catalyst.plans.logical.ExplainCommand import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -41,7 +40,11 @@ import org.apache.spark.sql.execution.QueryExecutionException import shark.LogHelper +//TODO work around for HiveContext, need to update that in Spark project (sql/hive), not here. case class CatalystContext(sc: SparkContext) extends HiveContext(sc) with LogHelper { + @transient protected[hive] override lazy val hiveconf = sessionState.getConf() + @transient protected[hive] override lazy val sessionState = SessionState.get() + class HiveQLQueryExecution(hql: String) extends QueryExecution { override def logical: LogicalPlan = HiveQl.parseSql(hql) override def toString = hql + "\n" + super.toString diff --git a/src/main/scala/shark/CatalystDriver.scala b/src/main/scala/shark/CatalystDriver.scala index 327d0f84..1e76cf65 100644 --- a/src/main/scala/shark/CatalystDriver.scala +++ b/src/main/scala/shark/CatalystDriver.scala @@ -1,16 +1,37 @@ +/* + * Copyright (C) 2012 The Regents of The University California. + * All rights reserved. + * + * Licensed 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 shark import java.util.ArrayList + +import scala.collection.JavaConversions._ + +import org.apache.commons.lang.exception.ExceptionUtils + +import org.apache.hive.service.cli.TableSchema + import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.metastore.api.Schema -import org.apache.hive.service.cli.TableSchema import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse -import org.apache.hadoop.hive.conf.HiveConf + import org.apache.spark.sql.hive.CatalystContext -import scala.collection.JavaConversions._ -import org.apache.commons.lang.exception.ExceptionUtils -class CatalystDriver(hconf: HiveConf) extends Driver with LogHelper { +class CatalystDriver extends Driver with LogHelper { private val context: CatalystContext = CatalystEnv.cc private var tschema: TableSchema = _ private var result: (Int, Seq[String], Throwable) = _ diff --git a/src/main/scala/shark/CatalystEnv.scala b/src/main/scala/shark/CatalystEnv.scala index 6a060272..5d69ba90 100755 --- a/src/main/scala/shark/CatalystEnv.scala +++ b/src/main/scala/shark/CatalystEnv.scala @@ -31,7 +31,7 @@ import org.apache.spark.scheduler.SplitInfo object CatalystEnv extends LogHelper { def init(): CatalystContext = { - if (sc == null) { + if (cc == null) { initWithCatalystContext() } diff --git a/src/main/scala/shark/SharkCliDriver.scala b/src/main/scala/shark/SharkCliDriver.scala index 7e8139b1..245ff75e 100755 --- a/src/main/scala/shark/SharkCliDriver.scala +++ b/src/main/scala/shark/SharkCliDriver.scala @@ -108,7 +108,7 @@ object SharkCliDriver { } catch { case e: UnsupportedEncodingException => System.exit(3) } - + if (!oproc.process_stage2(ss)) { System.exit(2) } @@ -163,6 +163,17 @@ object SharkCliDriver { val cli = new SharkCliDriver(reloadRdds) cli.setHiveVariables(oproc.getHiveVariables()) + // TODO work around for set the log output to console, because the HiveContext + // will set the output into an invalid buffer. + ss.in = System.in + try { + ss.out = new PrintStream(System.out, true, "UTF-8") + ss.info = new PrintStream(System.err, true, "UTF-8") + ss.err = new PrintStream(System.err, true, "UTF-8") + } catch { + case e: UnsupportedEncodingException => System.exit(3) + } + CatalystEnv.fixUncompatibleConf(conf) // Execute -i init files (always in silent mode) @@ -223,7 +234,7 @@ object SharkCliDriver { var prefix = "" val curDB = getFormattedDbMethod.invoke(null, conf, ss).asInstanceOf[String] - var curPrompt = SharkCliDriver.prompt + curDB + var curPrompt = SharkConfVars.getVar(conf, SharkConfVars.EXEC_MODE) + curDB var dbSpaces = spacesForStringMethod.invoke(null, curDB).asInstanceOf[String] line = reader.readLine(curPrompt + "> ") @@ -321,14 +332,11 @@ class SharkCliDriver(reloadRdds: Boolean = true) extends CliDriver with LogHelpe // hacking CommandProcessorFactory. val qp: Driver = if (SharkConfVars.getVar(conf, SharkConfVars.EXEC_MODE) == "catalyst") { - new CatalystDriver(hconf) + new CatalystDriver } else { proc.asInstanceOf[Driver] } - // TODO HiveContext shouldn't call the SessionState.start(), which has a conflict - // with CliSessionState. Here is a work around to set it(CliSessionState) back. - SessionState.start(ss) logInfo("Execution Mode: " + SharkConfVars.getVar(conf, SharkConfVars.EXEC_MODE)) qp.init() diff --git a/src/main/scala/shark/SharkServer2.scala b/src/main/scala/shark/SharkServer2.scala index e0042a52..3830f533 100644 --- a/src/main/scala/shark/SharkServer2.scala +++ b/src/main/scala/shark/SharkServer2.scala @@ -4,6 +4,7 @@ import org.apache.commons.logging.LogFactory import org.apache.hadoop.hive.common.LogUtils import org.apache.hadoop.hive.common.LogUtils.LogInitializationException import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService import org.apache.hive.service.server.{HiveServer2, ServerOptionsProcessor} import org.apache.hive.service.CompositeService @@ -12,11 +13,10 @@ import shark.server.SharkCLIService import scala.collection.JavaConversions._ -import org.apache.spark.SparkContext -import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.CatalystContext /** - * The main entry point for the Shark port of HiveServer2. Starts up a HiveContext and a SharkServer2 thrift server. + * The main entry point for the Shark port of HiveServer2. Starts up a CatalystContext and a SharkServer2 thrift server. */ object SharkServer2 extends Logging { var LOG = LogFactory.getLog(classOf[SharkServer2]) @@ -29,24 +29,29 @@ object SharkServer2 extends Logging { System.exit(-1) } + val ss = new SessionState(new HiveConf(classOf[SessionState])) + + // Set all properties specified via command line. + val hiveConf: HiveConf = ss.getConf() + + SessionState.start(ss) + logger.info("Starting SparkContext") - val sparkContext = new SparkContext("local", "") - logger.info("Starting HiveContext") - val hiveContext = new HiveContext(sparkContext) + CatalystEnv.init() + logger.info("Starting CatalystContext") //server.SharkServer.hiveContext = hiveContext Runtime.getRuntime.addShutdownHook( new Thread() { override def run() { - sparkContext.stop() + CatalystEnv.sc.stop() } } ) try { - val hiveConf = new HiveConf - val server = new SharkServer2(hiveContext) + val server = new SharkServer2(CatalystEnv.cc) server.init(hiveConf) server.start() logger.info("SharkServer2 started") @@ -59,9 +64,9 @@ object SharkServer2 extends Logging { } } -private[shark] class SharkServer2(hiveContext: HiveContext) extends HiveServer2 { +private[shark] class SharkServer2(catalystContext: CatalystContext) extends HiveServer2 { override def init(hiveConf: HiveConf): Unit = synchronized { - val sharkCLIService = new SharkCLIService(hiveContext) + val sharkCLIService = new SharkCLIService(catalystContext) Utils.setSuperField("cliService", sharkCLIService, this) addService(sharkCLIService) val sthriftCLIService = new ThriftBinaryCLIService(sharkCLIService) diff --git a/src/main/scala/shark/server/SharkCLIService.scala b/src/main/scala/shark/server/SharkCLIService.scala index ebbbe760..7d7808c7 100644 --- a/src/main/scala/shark/server/SharkCLIService.scala +++ b/src/main/scala/shark/server/SharkCLIService.scala @@ -8,15 +8,15 @@ import java.io.IOException import org.apache.hive.service.ServiceException import javax.security.auth.login.LoginException -import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.CatalystContext import shark.Utils -class SharkCLIService(hiveContext: HiveContext) extends CLIService { +class SharkCLIService(catalystContext: CatalystContext) extends CLIService { override def init(hiveConf: HiveConf) { this.synchronized { Utils.setSuperField("hiveConf", hiveConf, this) - val sharkSM = new SharkSessionManager(hiveContext) + val sharkSM = new SharkSessionManager(catalystContext) Utils.setSuperField("sessionManager", sharkSM, this) addService(sharkSM) try { From b6792db9b481dd36f076dbccf4ffbe833a117347 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 5 Jun 2014 22:13:20 +0800 Subject: [PATCH 17/21] remove the cached table reload for next PR --- src/main/scala/shark/SharkCliDriver.scala | 4 +- .../scala/shark/memstore2/CacheType.scala | 64 ----------------- .../shark/memstore2/SharkTblProperties.scala | 68 ------------------ .../scala/shark/memstore2/TableRecovery.scala | 69 ------------------- src/test/scala/shark/CliSuite.scala | 4 +- 5 files changed, 3 insertions(+), 206 deletions(-) delete mode 100644 src/main/scala/shark/memstore2/CacheType.scala delete mode 100644 src/main/scala/shark/memstore2/SharkTblProperties.scala delete mode 100644 src/main/scala/shark/memstore2/TableRecovery.scala diff --git a/src/main/scala/shark/SharkCliDriver.scala b/src/main/scala/shark/SharkCliDriver.scala index 245ff75e..517914d7 100755 --- a/src/main/scala/shark/SharkCliDriver.scala +++ b/src/main/scala/shark/SharkCliDriver.scala @@ -46,8 +46,6 @@ import org.apache.hadoop.hive.shims.ShimLoader import org.apache.hadoop.io.IOUtils import org.apache.thrift.transport.TSocket -import shark.memstore2.TableRecovery - object SharkCliDriver { val SKIP_RDD_RELOAD_FLAG = "-skipRddReload" @@ -292,7 +290,7 @@ class SharkCliDriver(reloadRdds: Boolean = true) extends CliDriver with LogHelpe console.printInfo( "Reloading cached RDDs from previous Shark sessions... (use %s flag to skip reloading)" .format(SharkCliDriver.SKIP_RDD_RELOAD_FLAG)) - TableRecovery.reloadRdds(processCmd(_), Some(console), ss) +// TableRecovery.reloadRdds(processCmd(_), Some(console), ss) } } diff --git a/src/main/scala/shark/memstore2/CacheType.scala b/src/main/scala/shark/memstore2/CacheType.scala deleted file mode 100644 index 2e9773ac..00000000 --- a/src/main/scala/shark/memstore2/CacheType.scala +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark.memstore2 - -import shark.LogHelper - - -/* - * Enumerations and static helper functions for caches supported by Shark. - */ -object CacheType extends Enumeration with LogHelper { - - /* - * The CacheTypes: - * - MEMORY: Stored in memory and on disk (i.e., cache is write-through). Persistent across Shark - * sessions. By default, all such tables are reloaded into memory on restart. - * - MEMORY_ONLY: Stored only in memory and dropped at the end of each Shark session. - * - OFFHEAP: Stored in an off-heap data storage format, specified by the System property - * 'shark.offheap.clientFactory'. Defaults to TachyonStorageClientFactory. - * - NONE: Stored on disk (e.g., HDFS) and managed by Hive. - */ - type CacheType = Value - val MEMORY, MEMORY_ONLY, OFFHEAP, NONE = Value - - def shouldCache(c: CacheType): Boolean = (c != NONE) - - /** Get the cache type object from a string representation. */ - def fromString(name: String): CacheType = Option(name).map(_.toUpperCase) match { - case None | Some("") | Some("FALSE") => NONE - case Some("TRUE") => MEMORY - case Some("HEAP") => - logWarning("The 'HEAP' cache type name is deprecated. Use 'MEMORY' instead.") - MEMORY - case Some("TACHYON") => - logWarning("The 'TACHYON' cache type name is deprecated. Use 'OFFHEAP' instead.") - OFFHEAP - case _ => { - try { - // Try to use Scala's Enumeration::withName() to interpret 'name'. - withName(name.toUpperCase) - } catch { - case e: java.util.NoSuchElementException => throw new InvalidCacheTypeException(name) - } - } - } - - class InvalidCacheTypeException(name: String) - extends Exception("Invalid string representation of cache type: '%s'".format(name)) -} diff --git a/src/main/scala/shark/memstore2/SharkTblProperties.scala b/src/main/scala/shark/memstore2/SharkTblProperties.scala deleted file mode 100644 index befc91d1..00000000 --- a/src/main/scala/shark/memstore2/SharkTblProperties.scala +++ /dev/null @@ -1,68 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark.memstore2 - -import java.util.{Map => JavaMap} - - -/** - * Collection of static fields and helpers for table properties (i.e., from A - * CREATE TABLE TBLPROPERTIES( ... ) used by Shark. - */ -object SharkTblProperties { - - case class TableProperty(varname: String, defaultVal: String) - - // Class name of the default cache policy used to manage partition evictions for cached, - // Hive-partitioned tables. - val CACHE_POLICY = new TableProperty("shark.cache.policy", "shark.memstore2.CacheAllPolicy") - - // Maximum size - in terms of the number of objects - of the cache specified by the - // "shark.cache.partition.cachePolicy" property above. - val MAX_PARTITION_CACHE_SIZE = new TableProperty("shark.cache.policy.maxSize", "10") - - // Default value for the "shark.cache" table property - val CACHE_FLAG = new TableProperty("shark.cache", "true") - - def getOrSetDefault(tblProps: JavaMap[String, String], variable: TableProperty): String = { - if (!tblProps.containsKey(variable.varname)) { - tblProps.put(variable.varname, variable.defaultVal) - } - tblProps.get(variable.varname) - } - - /** - * Returns value for the `variable` table property. If a value isn't present in `tblProps`, then - * the default for `variable` will be returned. - */ - def initializeWithDefaults( - tblProps: JavaMap[String, String], - isPartitioned: Boolean = false): JavaMap[String, String] = { - tblProps.put(CACHE_FLAG.varname, CACHE_FLAG.defaultVal) - if (isPartitioned) { - tblProps.put(CACHE_POLICY.varname, CACHE_POLICY.defaultVal) - } - tblProps - } - - def removeSharkProperties(tblProps: JavaMap[String, String]) { - tblProps.remove(CACHE_FLAG.varname) - tblProps.remove(CACHE_POLICY.varname) - tblProps.remove(MAX_PARTITION_CACHE_SIZE.varname) - } -} diff --git a/src/main/scala/shark/memstore2/TableRecovery.scala b/src/main/scala/shark/memstore2/TableRecovery.scala deleted file mode 100644 index faaa4fbf..00000000 --- a/src/main/scala/shark/memstore2/TableRecovery.scala +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark.memstore2 - -import java.util.{HashMap => JavaHashMap} - -import scala.collection.JavaConversions.asScalaBuffer - -import org.apache.hadoop.hive.ql.metadata.Hive -import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.hadoop.hive.cli.CliSessionState - -import shark.{LogHelper, CatalystEnv} -import shark.util.QueryRewriteUtils - -/** - * Singleton used to reload RDDs upon server restarts. - */ -object TableRecovery extends LogHelper { - - val db = Hive.get() - - /** - * Loads any cached tables with MEMORY as its `shark.cache` property. - * @param cmdRunner The runner that is responsible for taking a cached table query and - * a) Creating the table metadata in Hive Meta Store - * b) Loading the table as an RDD in memory - * @see SharkServer for an example usage. - * @param console Optional SessionState.LogHelper used, if present, to log information about - the tables that get reloaded. - */ - def reloadRdds(cmdRunner: String => Unit, console: Option[SessionState.LogHelper] = None, - ss: CliSessionState) { - // Filter for tables that should be reloaded into the cache. - val currentDbName = ss.getCurrentDatabase() - for (databaseName <- db.getAllDatabases(); tableName <- db.getAllTables(databaseName)) { - val hiveTable = db.getTable(databaseName, tableName) - val tblProps = hiveTable.getParameters - val cacheMode = CacheType.fromString(tblProps.get(SharkTblProperties.CACHE_FLAG.varname)) - if (cacheMode == CacheType.MEMORY) { - val logMessage = "Reloading %s.%s into memory.".format(databaseName, tableName) - if (console.isDefined) { - console.get.printInfo(logMessage) - } else { - logInfo(logMessage) - } - val cmd = QueryRewriteUtils.cacheToAlterTable("CACHE %s".format(tableName)) - cmdRunner(s"use $databaseName") - cmdRunner(cmd) - } - } - ss.setCurrentDatabase(currentDbName) - } -} diff --git a/src/test/scala/shark/CliSuite.scala b/src/test/scala/shark/CliSuite.scala index 5ead79bc..82d44647 100644 --- a/src/test/scala/shark/CliSuite.scala +++ b/src/test/scala/shark/CliSuite.scala @@ -52,8 +52,8 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with TestUtils { executeQuery("load data local inpath '" + dataFilePath+ "' overwrite into table shark_test1;") executeQuery("""create table shark_test1_cached TBLPROPERTIES ("shark.cache" = "true") as select * from shark_test1;""") - val out = executeQuery("select * from shark_test1_cached where key = 407;") - assert(out.contains("val_407")) + //val out = executeQuery("select * from shark_test1_cached where key = 407;") + //assert(out.contains("val_407")) } } From bf326ff3b08b49152b454e0875ce702e4da26a9c Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 5 Jun 2014 23:17:33 +0800 Subject: [PATCH 18/21] Minimize the changes for SharkBuild.scala --- project/SharkBuild.scala | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index bd2d72dc..285d037c 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -32,13 +32,12 @@ object SharkBuild extends Build { val SHARK_ORGANIZATION = "edu.berkeley.cs.shark" - val SPARK_VERSION = "1.1.0-SNAPSHOT" + val SPARK_VERSION = "1.0.0-SNAPSHOT" val SCALA_VERSION = "2.10.4" val SCALAC_JVM_VERSION = "jvm-1.6" val JAVAC_JVM_VERSION = "1.6" - val JETTY_VERSION = "8.1.14.v20131031" // Hadoop version to build against. For example, "0.20.2", "0.20.205.0", or // "1.0.1" for Apache releases, or "0.20.2-cdh3u3" for Cloudera Hadoop. @@ -72,7 +71,7 @@ object SharkBuild extends Build { // Exclusion rules for Hive artifacts val excludeGuava = ExclusionRule(organization = "com.google.guava") val excludeLog4j = ExclusionRule(organization = "log4j") - val excludeJetty = ExclusionRule(organization = "org.mortbay.jetty") + val excludeServlet = ExclusionRule(organization = "org.mortbay.jetty") val excludeXerces = ExclusionRule(organization = "xerces") val excludeHive = ExclusionRule(organization = "org.apache.hive") @@ -81,7 +80,7 @@ object SharkBuild extends Build { val hiveDependencies = hiveArtifacts.map ( artifactId => "org.spark-project.hive" % artifactId % "0.12.0" excludeAll( - excludeGuava, excludeLog4j, excludeAsm, excludeJetty, excludeNetty, excludeXerces) + excludeGuava, excludeLog4j, excludeAsm, excludeNetty, excludeXerces, excludeServlet) ) val yarnDependency = (if (YARN_ENABLED) { @@ -102,13 +101,11 @@ object SharkBuild extends Build { libraryDependencies ++= hiveDependencies ++ yarnDependency, libraryDependencies ++= Seq( - "org.apache.spark" %% "spark-hive" % SPARK_VERSION excludeAll(excludeHive, excludeJetty) force(), + "org.apache.spark" %% "spark-hive" % SPARK_VERSION excludeAll(excludeHive, excludeServlet) force(), "org.apache.spark" %% "spark-repl" % SPARK_VERSION, "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm) force(), "org.mortbay.jetty" % "jetty" % "6.1.26" exclude ("org.mortbay.jetty", "servlet-api") force(), "org.eclipse.jetty.orbit" % "javax.servlet" % "3.0.0.v201112011016" artifacts ( Artifact("javax.servlet", "jar", "jar") ), - "com.google.guava" % "guava" % "14.0.1", - "commons-io" % "commons-io" % "2.1", "com.typesafe" %% "scalalogging-slf4j" % "1.0.1", "org.scalatest" %% "scalatest" % "1.9.1" % "test" ), From a3732b911dd6f997dfa2a521aadbca69baacf6de Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 5 Jun 2014 23:33:00 +0800 Subject: [PATCH 19/21] Put the local maven as the last resolver --- project/SharkBuild.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index 285d037c..19894ee8 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -113,12 +113,12 @@ object SharkBuild extends Build { // Download managed jars into lib_managed. retrieveManaged := true, resolvers ++= Seq( - "Local Maven" at Path.userHome.asFile.toURI.toURL + ".m2/repository", "Maven Repository" at "http://repo.maven.apache.org/maven2", "Apache Repository" at "https://repository.apache.org/content/repositories/releases", "JBoss Repository" at "https://repository.jboss.org/nexus/content/repositories/releases/", "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/", - "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/" + "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/", + "Local Maven" at Path.userHome.asFile.toURI.toURL + ".m2/repository" ), publishTo <<= version { (v: String) => From 02652cfa164472ea9aa26a5a63a21141478789da Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 12 Jun 2014 15:22:51 +0800 Subject: [PATCH 20/21] remove the unused class --- .../scala/shark/util/QueryRewriteUtils.scala | 50 ------------------- 1 file changed, 50 deletions(-) delete mode 100644 src/main/scala/shark/util/QueryRewriteUtils.scala diff --git a/src/main/scala/shark/util/QueryRewriteUtils.scala b/src/main/scala/shark/util/QueryRewriteUtils.scala deleted file mode 100644 index de43ca01..00000000 --- a/src/main/scala/shark/util/QueryRewriteUtils.scala +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Copyright (C) 2012 The Regents of The University California. - * All rights reserved. - * - * Licensed 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 shark.util - -import org.apache.hadoop.hive.ql.parse.SemanticException - -object QueryRewriteUtils { - - def cacheToAlterTable(cmd: String): String = { - val CACHE_TABLE_DEFAULT = "(?i)CACHE ([^ ]+)".r - val CACHE_TABLE_IN = "(?i)CACHE ([^ ]+) IN ([^ ]+)".r - - cmd match { - case CACHE_TABLE_DEFAULT(tableName) => - s"ALTER TABLE $tableName SET TBLPROPERTIES ('shark.cache' = 'memory')" - case CACHE_TABLE_IN(tableName, cacheType) => - s"ALTER TABLE $tableName SET TBLPROPERTIES ('shark.cache' = '$cacheType')" - case _ => - throw new SemanticException( - s"CACHE accepts a single table name: 'CACHE
[IN ]'" + - s" (received command: '$cmd')") - } - } - - def uncacheToAlterTable(cmd: String): String = { - val cmdSplit = cmd.split(' ') - if (cmdSplit.size == 2) { - val tableName = cmdSplit(1) - "ALTER TABLE %s SET TBLPROPERTIES ('shark.cache' = 'false')".format(tableName) - } else { - throw new SemanticException( - s"UNCACHE accepts a single table name: 'UNCACHE
' (received command: '$cmd')") - } - } -} From 34706791102b37d05bbf2266d387856805694076 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 12 Jun 2014 15:24:15 +0800 Subject: [PATCH 21/21] Make the unittest work --- project/SharkBuild.scala | 2 +- src/main/scala/shark/CatalystContext.scala | 3 +-- src/main/scala/shark/CatalystDriver.scala | 3 +-- src/main/scala/shark/SharkServer2.scala | 1 + 4 files changed, 4 insertions(+), 5 deletions(-) diff --git a/project/SharkBuild.scala b/project/SharkBuild.scala index 19894ee8..1c474e8a 100755 --- a/project/SharkBuild.scala +++ b/project/SharkBuild.scala @@ -32,7 +32,7 @@ object SharkBuild extends Build { val SHARK_ORGANIZATION = "edu.berkeley.cs.shark" - val SPARK_VERSION = "1.0.0-SNAPSHOT" + val SPARK_VERSION = "1.1.0-SNAPSHOT" val SCALA_VERSION = "2.10.4" diff --git a/src/main/scala/shark/CatalystContext.scala b/src/main/scala/shark/CatalystContext.scala index 0852ca46..cd2f5795 100644 --- a/src/main/scala/shark/CatalystContext.scala +++ b/src/main/scala/shark/CatalystContext.scala @@ -55,8 +55,7 @@ case class CatalystContext(sc: SparkContext) extends HiveContext(sc) with LogHel */ def result(): (Int, Seq[String], Throwable) = analyzed match { case NativeCommand(cmd) => runOnHive(cmd) - case ExplainCommand(plan) => - (0, new QueryExecution { val logical = plan }.toString.split("\n"), null) + case ExplainCommand(plan) => (0, executePlan(plan).toString.split("\n"), null) case query => try{ val result: Seq[Seq[Any]] = toRdd.collect().toSeq diff --git a/src/main/scala/shark/CatalystDriver.scala b/src/main/scala/shark/CatalystDriver.scala index 1e76cf65..8bca701c 100644 --- a/src/main/scala/shark/CatalystDriver.scala +++ b/src/main/scala/shark/CatalystDriver.scala @@ -31,8 +31,7 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse import org.apache.spark.sql.hive.CatalystContext -class CatalystDriver extends Driver with LogHelper { - private val context: CatalystContext = CatalystEnv.cc +class CatalystDriver(val context: CatalystContext = CatalystEnv.cc) extends Driver with LogHelper { private var tschema: TableSchema = _ private var result: (Int, Seq[String], Throwable) = _ diff --git a/src/main/scala/shark/SharkServer2.scala b/src/main/scala/shark/SharkServer2.scala index 3830f533..95ef5668 100644 --- a/src/main/scala/shark/SharkServer2.scala +++ b/src/main/scala/shark/SharkServer2.scala @@ -39,6 +39,7 @@ object SharkServer2 extends Logging { logger.info("Starting SparkContext") CatalystEnv.init() logger.info("Starting CatalystContext") + SessionState.start(ss) //server.SharkServer.hiveContext = hiveContext