From d6fb485de8b79054db08658d904a3148a04d4180 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 18 Apr 2016 10:13:38 -0700 Subject: [PATCH 01/69] [SPARK-14423][YARN] Avoid same name files added to distributed cache again ## What changes were proposed in this pull request? In the current implementation of assembly-free spark deployment, jars under `assembly/target/scala-xxx/jars` will be uploaded to distributed cache by default, there's a chance these jars' name will be conflicted with name of jars specified in `--jars`, this will introduce exception when starting application: ``` client token: N/A diagnostics: Application application_1459907402325_0004 failed 2 times due to AM Container for appattempt_1459907402325_0004_000002 exited with exitCode: -1000 For more detailed output, check application tracking page:http://hw12100.local:8088/proxy/application_1459907402325_0004/Then, click on links to logs of each attempt. Diagnostics: Resource hdfs://localhost:8020/user/sshao/.sparkStaging/application_1459907402325_0004/avro-mapred-1.7.7-hadoop2.jar changed on src filesystem (expected 1459909780508, was 1459909782590 java.io.IOException: Resource hdfs://localhost:8020/user/sshao/.sparkStaging/application_1459907402325_0004/avro-mapred-1.7.7-hadoop2.jar changed on src filesystem (expected 1459909780508, was 1459909782590 at org.apache.hadoop.yarn.util.FSDownload.copy(FSDownload.java:253) at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:61) at org.apache.hadoop.yarn.util.FSDownload$2.run(FSDownload.java:359) at org.apache.hadoop.yarn.util.FSDownload$2.run(FSDownload.java:357) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628) at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:356) at org.apache.hadoop.yarn.util.FSDownload.call(FSDownload.java:60) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) ``` So here by checking the name of file to avoid same name files uploaded again. ## How was this patch tested? Unit test and manual integrated test is done locally. Author: jerryshao Closes #12203 from jerryshao/SPARK-14423. --- .../org/apache/spark/deploy/yarn/Client.scala | 14 ++++++-- .../spark/deploy/yarn/ClientSuite.scala | 32 ++++++++++++++++++- 2 files changed, 42 insertions(+), 4 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 04e91f8553d51..7c168ed279df6 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -364,6 +364,10 @@ private[spark] class Client( // multiple times, YARN will fail to launch containers for the app with an internal // error. val distributedUris = new HashSet[String] + // Used to keep track of URIs(files) added to the distribute cache have the same name. If + // same name but different path files are added multiple time, YARN will fail to launch + // containers for the app with an internal error. + val distributedNames = new HashSet[String] YarnSparkHadoopUtil.get.obtainTokenForHiveMetastore(sparkConf, hadoopConf, credentials) YarnSparkHadoopUtil.get.obtainTokenForHBase(sparkConf, hadoopConf, credentials) @@ -376,11 +380,16 @@ private[spark] class Client( def addDistributedUri(uri: URI): Boolean = { val uriStr = uri.toString() + val fileName = new File(uri.getPath).getName if (distributedUris.contains(uriStr)) { - logWarning(s"Resource $uri added multiple times to distributed cache.") + logWarning(s"Same path resource $uri added multiple times to distributed cache.") + false + } else if (distributedNames.contains(fileName)) { + logWarning(s"Same name resource $uri added multiple times to distributed cache") false } else { distributedUris += uriStr + distributedNames += fileName true } } @@ -519,8 +528,7 @@ private[spark] class Client( ).foreach { case (flist, resType, addToClasspath) => flist.foreach { file => val (_, localizedPath) = distribute(file, resType = resType) - require(localizedPath != null) - if (addToClasspath) { + if (addToClasspath && localizedPath != null) { cachedSecondaryJarLinks += localizedPath } } diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index 74e268dc48473..23050e8c1d5c1 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.yarn -import java.io.{File, FileOutputStream} +import java.io.{File, FileInputStream, FileOutputStream} import java.net.URI import java.util.Properties @@ -285,6 +285,36 @@ class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll classpath(client) should contain (buildPath(PWD, LOCALIZED_LIB_DIR, "*")) } + test("ignore same name jars") { + val libs = Utils.createTempDir() + val jarsDir = new File(libs, "jars") + assert(jarsDir.mkdir()) + new FileOutputStream(new File(libs, "RELEASE")).close() + val userLibs = Utils.createTempDir() + + val jar1 = TestUtils.createJarWithFiles(Map(), jarsDir) + val jar2 = TestUtils.createJarWithFiles(Map(), userLibs) + // Copy jar2 to jar3 with same name + val jar3 = { + val target = new File(userLibs, new File(jar1.toURI).getName) + val input = new FileInputStream(jar2.getPath) + val output = new FileOutputStream(target) + Utils.copyStream(input, output, closeStreams = true) + target.toURI.toURL + } + + val sparkConf = new SparkConfWithEnv(Map("SPARK_HOME" -> libs.getAbsolutePath)) + .set(JARS_TO_DISTRIBUTE, Seq(jar2.getPath, jar3.getPath)) + + val client = createClient(sparkConf) + val tempDir = Utils.createTempDir() + client.prepareLocalResources(tempDir.getAbsolutePath(), Nil) + + // Only jar2 will be added to SECONDARY_JARS, jar3 which has the same name with jar1 will be + // ignored. + sparkConf.get(SECONDARY_JARS) should be (Some(Seq(new File(jar2.toURI).getName))) + } + object Fixtures { val knownDefYarnAppCP: Seq[String] = From 432d1399cb6985893932088875b2f3be981c0b5f Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 18 Apr 2016 10:44:51 -0700 Subject: [PATCH 02/69] [SPARK-14614] [SQL] Add `bround` function ## What changes were proposed in this pull request? This PR aims to add `bound` function (aka Banker's round) by extending current `round` implementation. [Hive supports `bround` since 1.3.0.](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF) **Hive (1.3 ~ 2.0)** ``` hive> select round(2.5), bround(2.5); OK 3.0 2.0 ``` **After this PR** ```scala scala> sql("select round(2.5), bround(2.5)").head res0: org.apache.spark.sql.Row = [3,2] ``` ## How was this patch tested? Pass the Jenkins tests (with extended tests). Author: Dongjoon Hyun Closes #12376 from dongjoon-hyun/SPARK-14614. --- .../catalyst/analysis/FunctionRegistry.scala | 1 + .../expressions/mathExpressions.scala | 71 ++++++++++++------- .../org/apache/spark/sql/types/Decimal.scala | 6 ++ .../ExpressionTypeCheckingSuite.scala | 10 ++- .../expressions/MathFunctionsSuite.scala | 23 +++++- .../org/apache/spark/sql/functions.scala | 17 +++++ .../spark/sql/MathExpressionsSuite.scala | 12 +++- 7 files changed, 113 insertions(+), 27 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 028463ed4f60f..ed19191b72f0e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -179,6 +179,7 @@ object FunctionRegistry { expression[Atan]("atan"), expression[Atan2]("atan2"), expression[Bin]("bin"), + expression[BRound]("bround"), expression[Cbrt]("cbrt"), expression[Ceil]("ceil"), expression[Ceil]("ceiling"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala index c8a28e847745c..9e190289b7db0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala @@ -779,7 +779,6 @@ case class Logarithm(left: Expression, right: Expression) /** * Round the `child`'s result to `scale` decimal place when `scale` >= 0 * or round at integral part when `scale` < 0. - * For example, round(31.415, 2) = 31.42 and round(31.415, -1) = 30. * * Child of IntegralType would round to itself when `scale` >= 0. * Child of FractionalType whose value is NaN or Infinite would always round to itself. @@ -789,16 +788,12 @@ case class Logarithm(left: Expression, right: Expression) * * @param child expr to be round, all [[NumericType]] is allowed as Input * @param scale new scale to be round to, this should be a constant int at runtime + * @param mode rounding mode (e.g. HALF_UP, HALF_UP) + * @param modeStr rounding mode string name (e.g. "ROUND_HALF_UP", "ROUND_HALF_EVEN") */ -@ExpressionDescription( - usage = "_FUNC_(x, d) - Round x to d decimal places.", - extended = "> SELECT _FUNC_(12.3456, 1);\n 12.3") -case class Round(child: Expression, scale: Expression) - extends BinaryExpression with ImplicitCastInputTypes { - - import BigDecimal.RoundingMode.HALF_UP - - def this(child: Expression) = this(child, Literal(0)) +abstract class RoundBase(child: Expression, scale: Expression, + mode: BigDecimal.RoundingMode.Value, modeStr: String) + extends BinaryExpression with Serializable with ImplicitCastInputTypes { override def left: Expression = child override def right: Expression = scale @@ -853,28 +848,28 @@ case class Round(child: Expression, scale: Expression) child.dataType match { case _: DecimalType => val decimal = input1.asInstanceOf[Decimal] - if (decimal.changePrecision(decimal.precision, _scale)) decimal else null + if (decimal.changePrecision(decimal.precision, _scale, mode)) decimal else null case ByteType => - BigDecimal(input1.asInstanceOf[Byte]).setScale(_scale, HALF_UP).toByte + BigDecimal(input1.asInstanceOf[Byte]).setScale(_scale, mode).toByte case ShortType => - BigDecimal(input1.asInstanceOf[Short]).setScale(_scale, HALF_UP).toShort + BigDecimal(input1.asInstanceOf[Short]).setScale(_scale, mode).toShort case IntegerType => - BigDecimal(input1.asInstanceOf[Int]).setScale(_scale, HALF_UP).toInt + BigDecimal(input1.asInstanceOf[Int]).setScale(_scale, mode).toInt case LongType => - BigDecimal(input1.asInstanceOf[Long]).setScale(_scale, HALF_UP).toLong + BigDecimal(input1.asInstanceOf[Long]).setScale(_scale, mode).toLong case FloatType => val f = input1.asInstanceOf[Float] if (f.isNaN || f.isInfinite) { f } else { - BigDecimal(f.toDouble).setScale(_scale, HALF_UP).toFloat + BigDecimal(f.toDouble).setScale(_scale, mode).toFloat } case DoubleType => val d = input1.asInstanceOf[Double] if (d.isNaN || d.isInfinite) { d } else { - BigDecimal(d).setScale(_scale, HALF_UP).toDouble + BigDecimal(d).setScale(_scale, mode).toDouble } } } @@ -885,7 +880,8 @@ case class Round(child: Expression, scale: Expression) val evaluationCode = child.dataType match { case _: DecimalType => s""" - if (${ce.value}.changePrecision(${ce.value}.precision(), ${_scale})) { + if (${ce.value}.changePrecision(${ce.value}.precision(), ${_scale}, + java.math.BigDecimal.${modeStr})) { ${ev.value} = ${ce.value}; } else { ${ev.isNull} = true; @@ -894,7 +890,7 @@ case class Round(child: Expression, scale: Expression) if (_scale < 0) { s""" ${ev.value} = new java.math.BigDecimal(${ce.value}). - setScale(${_scale}, java.math.BigDecimal.ROUND_HALF_UP).byteValue();""" + setScale(${_scale}, java.math.BigDecimal.${modeStr}).byteValue();""" } else { s"${ev.value} = ${ce.value};" } @@ -902,7 +898,7 @@ case class Round(child: Expression, scale: Expression) if (_scale < 0) { s""" ${ev.value} = new java.math.BigDecimal(${ce.value}). - setScale(${_scale}, java.math.BigDecimal.ROUND_HALF_UP).shortValue();""" + setScale(${_scale}, java.math.BigDecimal.${modeStr}).shortValue();""" } else { s"${ev.value} = ${ce.value};" } @@ -910,7 +906,7 @@ case class Round(child: Expression, scale: Expression) if (_scale < 0) { s""" ${ev.value} = new java.math.BigDecimal(${ce.value}). - setScale(${_scale}, java.math.BigDecimal.ROUND_HALF_UP).intValue();""" + setScale(${_scale}, java.math.BigDecimal.${modeStr}).intValue();""" } else { s"${ev.value} = ${ce.value};" } @@ -918,7 +914,7 @@ case class Round(child: Expression, scale: Expression) if (_scale < 0) { s""" ${ev.value} = new java.math.BigDecimal(${ce.value}). - setScale(${_scale}, java.math.BigDecimal.ROUND_HALF_UP).longValue();""" + setScale(${_scale}, java.math.BigDecimal.${modeStr}).longValue();""" } else { s"${ev.value} = ${ce.value};" } @@ -928,7 +924,7 @@ case class Round(child: Expression, scale: Expression) ${ev.value} = ${ce.value}; } else { ${ev.value} = java.math.BigDecimal.valueOf(${ce.value}). - setScale(${_scale}, java.math.BigDecimal.ROUND_HALF_UP).floatValue(); + setScale(${_scale}, java.math.BigDecimal.${modeStr}).floatValue(); }""" case DoubleType => // if child eval to NaN or Infinity, just return it. s""" @@ -936,7 +932,7 @@ case class Round(child: Expression, scale: Expression) ${ev.value} = ${ce.value}; } else { ${ev.value} = java.math.BigDecimal.valueOf(${ce.value}). - setScale(${_scale}, java.math.BigDecimal.ROUND_HALF_UP).doubleValue(); + setScale(${_scale}, java.math.BigDecimal.${modeStr}).doubleValue(); }""" } @@ -957,3 +953,30 @@ case class Round(child: Expression, scale: Expression) } } } + +/** + * Round an expression to d decimal places using HALF_UP rounding mode. + * round(2.5) == 3.0, round(3.5) == 4.0. + */ +@ExpressionDescription( + usage = "_FUNC_(x, d) - Round x to d decimal places using HALF_UP rounding mode.", + extended = "> SELECT _FUNC_(2.5, 0);\n 3.0") +case class Round(child: Expression, scale: Expression) + extends RoundBase(child, scale, BigDecimal.RoundingMode.HALF_UP, "ROUND_HALF_UP") + with Serializable with ImplicitCastInputTypes { + def this(child: Expression) = this(child, Literal(0)) +} + +/** + * Round an expression to d decimal places using HALF_EVEN rounding mode, + * also known as Gaussian rounding or bankers' rounding. + * round(2.5) = 2.0, round(3.5) = 4.0. + */ +@ExpressionDescription( + usage = "_FUNC_(x, d) - Round x to d decimal places using HALF_EVEN rounding mode.", + extended = "> SELECT _FUNC_(2.5, 0);\n 2.0") +case class BRound(child: Expression, scale: Expression) + extends RoundBase(child, scale, BigDecimal.RoundingMode.HALF_EVEN, "ROUND_HALF_EVEN") + with Serializable with ImplicitCastInputTypes { + def this(child: Expression) = this(child, Literal(0)) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index a30a3926bb86e..6f4ec6b701919 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -201,6 +201,11 @@ final class Decimal extends Ordered[Decimal] with Serializable { changePrecision(precision, scale, ROUND_HALF_UP) } + def changePrecision(precision: Int, scale: Int, mode: Int): Boolean = mode match { + case java.math.BigDecimal.ROUND_HALF_UP => changePrecision(precision, scale, ROUND_HALF_UP) + case java.math.BigDecimal.ROUND_HALF_EVEN => changePrecision(precision, scale, ROUND_HALF_EVEN) + } + /** * Update precision and scale while keeping our value the same, and return true if successful. * @@ -337,6 +342,7 @@ final class Decimal extends Ordered[Decimal] with Serializable { object Decimal { val ROUND_HALF_UP = BigDecimal.RoundingMode.HALF_UP + val ROUND_HALF_EVEN = BigDecimal.RoundingMode.HALF_EVEN val ROUND_CEILING = BigDecimal.RoundingMode.CEILING val ROUND_FLOOR = BigDecimal.RoundingMode.FLOOR diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala index ace6e10c6ec30..660dc86c3e284 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala @@ -192,7 +192,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite { "values of function map should all be the same type") } - test("check types for ROUND") { + test("check types for ROUND/BROUND") { assertSuccess(Round(Literal(null), Literal(null))) assertSuccess(Round('intField, Literal(1))) @@ -200,6 +200,14 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite { assertError(Round('intField, 'booleanField), "requires int type") assertError(Round('intField, 'mapField), "requires int type") assertError(Round('booleanField, 'intField), "requires numeric type") + + assertSuccess(BRound(Literal(null), Literal(null))) + assertSuccess(BRound('intField, Literal(1))) + + assertError(BRound('intField, 'intField), "Only foldable Expression is allowed") + assertError(BRound('intField, 'booleanField), "requires int type") + assertError(BRound('intField, 'mapField), "requires int type") + assertError(BRound('booleanField, 'intField), "requires numeric type") } test("check types for Greatest/Least") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala index 452792d21c204..1e5b657f1fb98 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala @@ -508,7 +508,7 @@ class MathFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkConsistencyBetweenInterpretedAndCodegen(Logarithm, DoubleType, DoubleType) } - test("round") { + test("round/bround") { val scales = -6 to 6 val doublePi: Double = math.Pi val shortPi: Short = 31415 @@ -529,11 +529,18 @@ class MathFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { 31415926535900000L, 31415926535898000L, 31415926535897900L, 31415926535897930L) ++ Seq.fill(7)(31415926535897932L) + val intResultsB: Seq[Int] = Seq(314000000, 314200000, 314160000, 314159000, 314159300, + 314159260) ++ Seq.fill(7)(314159265) + scales.zipWithIndex.foreach { case (scale, i) => checkEvaluation(Round(doublePi, scale), doubleResults(i), EmptyRow) checkEvaluation(Round(shortPi, scale), shortResults(i), EmptyRow) checkEvaluation(Round(intPi, scale), intResults(i), EmptyRow) checkEvaluation(Round(longPi, scale), longResults(i), EmptyRow) + checkEvaluation(BRound(doublePi, scale), doubleResults(i), EmptyRow) + checkEvaluation(BRound(shortPi, scale), shortResults(i), EmptyRow) + checkEvaluation(BRound(intPi, scale), intResultsB(i), EmptyRow) + checkEvaluation(BRound(longPi, scale), longResults(i), EmptyRow) } val bdResults: Seq[BigDecimal] = Seq(BigDecimal(3.0), BigDecimal(3.1), BigDecimal(3.14), @@ -543,19 +550,33 @@ class MathFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { // and not allowed by o.a.s.s.types.Decimal.changePrecision, therefore null (0 to 7).foreach { i => checkEvaluation(Round(bdPi, i), bdResults(i), EmptyRow) + checkEvaluation(BRound(bdPi, i), bdResults(i), EmptyRow) } (8 to 10).foreach { scale => checkEvaluation(Round(bdPi, scale), null, EmptyRow) + checkEvaluation(BRound(bdPi, scale), null, EmptyRow) } DataTypeTestUtils.numericTypes.foreach { dataType => checkEvaluation(Round(Literal.create(null, dataType), Literal(2)), null) checkEvaluation(Round(Literal.create(null, dataType), Literal.create(null, IntegerType)), null) + checkEvaluation(BRound(Literal.create(null, dataType), Literal(2)), null) + checkEvaluation(BRound(Literal.create(null, dataType), + Literal.create(null, IntegerType)), null) } + checkEvaluation(Round(2.5, 0), 3.0) + checkEvaluation(Round(3.5, 0), 4.0) + checkEvaluation(Round(-2.5, 0), -3.0) checkEvaluation(Round(-3.5, 0), -4.0) checkEvaluation(Round(-0.35, 1), -0.4) checkEvaluation(Round(-35, -1), -40) + checkEvaluation(BRound(2.5, 0), 2.0) + checkEvaluation(BRound(3.5, 0), 4.0) + checkEvaluation(BRound(-2.5, 0), -2.0) + checkEvaluation(BRound(-3.5, 0), -4.0) + checkEvaluation(BRound(-0.35, 1), -0.4) + checkEvaluation(BRound(-35, -1), -40) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 223122300dbb3..8e2e94669b8c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -1776,6 +1776,23 @@ object functions { */ def round(e: Column, scale: Int): Column = withExpr { Round(e.expr, Literal(scale)) } + /** + * Returns the value of the column `e` rounded to 0 decimal places with HALF_EVEN round mode. + * + * @group math_funcs + * @since 2.0.0 + */ + def bround(e: Column): Column = bround(e, 0) + + /** + * Round the value of `e` to `scale` decimal places with HALF_EVEN round mode + * if `scale` >= 0 or at integral part when `scale` < 0. + * + * @group math_funcs + * @since 2.0.0 + */ + def bround(e: Column, scale: Int): Column = withExpr { BRound(e.expr, Literal(scale)) } + /** * Shift the given value numBits left. If the given value is a long value, this function * will return a long value else it will return an integer value. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala index f5a67fd782d63..0de7f2321f398 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala @@ -207,12 +207,16 @@ class MathExpressionsSuite extends QueryTest with SharedSQLContext { testOneToOneMathFunction(rint, math.rint) } - test("round") { + test("round/bround") { val df = Seq(5, 55, 555).map(Tuple1(_)).toDF("a") checkAnswer( df.select(round('a), round('a, -1), round('a, -2)), Seq(Row(5, 10, 0), Row(55, 60, 100), Row(555, 560, 600)) ) + checkAnswer( + df.select(bround('a), bround('a, -1), bround('a, -2)), + Seq(Row(5, 0, 0), Row(55, 60, 100), Row(555, 560, 600)) + ) val pi = "3.1415" checkAnswer( @@ -221,6 +225,12 @@ class MathExpressionsSuite extends QueryTest with SharedSQLContext { Seq(Row(BigDecimal("0E3"), BigDecimal("0E2"), BigDecimal("0E1"), BigDecimal(3), BigDecimal("3.1"), BigDecimal("3.14"), BigDecimal("3.142"))) ) + checkAnswer( + sql(s"SELECT bround($pi, -3), bround($pi, -2), bround($pi, -1), " + + s"bround($pi, 0), bround($pi, 1), bround($pi, 2), bround($pi, 3)"), + Seq(Row(BigDecimal("0E3"), BigDecimal("0E2"), BigDecimal("0E1"), BigDecimal(3), + BigDecimal("3.1"), BigDecimal("3.14"), BigDecimal("3.142"))) + ) } test("exp") { From 775cf17eaaae1a38efe47b282b1d6bbdb99bd759 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 18 Apr 2016 11:09:33 -0700 Subject: [PATCH 03/69] [SPARK-14473][SQL] Define analysis rules to catch operations not supported in streaming ## What changes were proposed in this pull request? There are many operations that are currently not supported in the streaming execution. For example: - joining two streams - unioning a stream and a batch source - sorting - window functions (not time windows) - distinct aggregates Furthermore, executing a query with a stream source as a batch query should also fail. This patch add an additional step after analysis in the QueryExecution which will check that all the operations in the analyzed logical plan is supported or not. ## How was this patch tested? unit tests. Author: Tathagata Das Closes #12246 from tdas/SPARK-14473. --- .../sql/catalyst/analysis/OutputMode.scala | 23 ++ .../UnsupportedOperationChecker.scala | 145 +++++++ .../spark/sql/catalyst/dsl/package.scala | 18 + .../catalyst/plans/logical/LogicalPlan.scala | 3 + .../analysis/UnsupportedOperationsSuite.scala | 384 ++++++++++++++++++ .../sql/catalyst/plans/LogicalPlanSuite.scala | 21 + .../spark/sql/ContinuousQueryManager.scala | 16 +- .../scala/org/apache/spark/sql/Dataset.scala | 4 +- .../spark/sql/execution/QueryExecution.scala | 9 + .../streaming/IncrementalExecution.scala | 8 +- .../execution/streaming/StreamExecution.scala | 11 +- .../streaming/StreamingRelation.scala | 2 + .../apache/spark/sql/internal/SQLConf.scala | 8 + .../org/apache/spark/sql/StreamTest.scala | 8 +- .../spark/sql/streaming/StreamSuite.scala | 34 +- .../streaming/StreamingAggregationSuite.scala | 3 + 16 files changed, 685 insertions(+), 12 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/OutputMode.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/OutputMode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/OutputMode.scala new file mode 100644 index 0000000000000..a4d387eae3c80 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/OutputMode.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.analysis + +sealed trait OutputMode + +case object Append extends OutputMode +case object Update extends OutputMode diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala new file mode 100644 index 0000000000000..aadc1d31bd4b2 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.analysis + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical._ + +/** + * Analyzes the presence of unsupported operations in a logical plan. + */ +object UnsupportedOperationChecker { + + def checkForBatch(plan: LogicalPlan): Unit = { + plan.foreachUp { + case p if p.isStreaming => + throwError( + "Queries with streaming sources must be executed with write.startStream()")(p) + + case _ => + } + } + + def checkForStreaming(plan: LogicalPlan, outputMode: OutputMode): Unit = { + + if (!plan.isStreaming) { + throwError( + "Queries without streaming sources cannot be executed with write.startStream()")(plan) + } + + plan.foreachUp { implicit plan => + + // Operations that cannot exists anywhere in a streaming plan + plan match { + + case _: Command => + throwError("Commands like CreateTable*, AlterTable*, Show* are not supported with " + + "streaming DataFrames/Datasets") + + case _: InsertIntoTable => + throwError("InsertIntoTable is not supported with streaming DataFrames/Datasets") + + case Aggregate(_, _, child) if child.isStreaming && outputMode == Append => + throwError( + "Aggregations are not supported on streaming DataFrames/Datasets in " + + "Append output mode. Consider changing output mode to Update.") + + case Join(left, right, joinType, _) => + + joinType match { + + case Inner => + if (left.isStreaming && right.isStreaming) { + throwError("Inner join between two streaming DataFrames/Datasets is not supported") + } + + case FullOuter => + if (left.isStreaming || right.isStreaming) { + throwError("Full outer joins with streaming DataFrames/Datasets are not supported") + } + + + case LeftOuter | LeftSemi | LeftAnti => + if (right.isStreaming) { + throwError("Left outer/semi/anti joins with a streaming DataFrame/Dataset " + + "on the right is not supported") + } + + case RightOuter => + if (left.isStreaming) { + throwError("Right outer join with a streaming DataFrame/Dataset on the left is " + + "not supported") + } + + case NaturalJoin(_) | UsingJoin(_, _) => + // They should not appear in an analyzed plan. + + case _ => + throwError(s"Join type $joinType is not supported with streaming DataFrame/Dataset") + } + + case c: CoGroup if c.children.exists(_.isStreaming) => + throwError("CoGrouping with a streaming DataFrame/Dataset is not supported") + + case u: Union if u.children.map(_.isStreaming).distinct.size == 2 => + throwError("Union between streaming and batch DataFrames/Datasets is not supported") + + case Except(left, right) if right.isStreaming => + throwError("Except with a streaming DataFrame/Dataset on the right is not supported") + + case Intersect(left, right) if left.isStreaming && right.isStreaming => + throwError("Intersect between two streaming DataFrames/Datasets is not supported") + + case GroupingSets(_, _, child, _) if child.isStreaming => + throwError("GroupingSets is not supported on streaming DataFrames/Datasets") + + case GlobalLimit(_, _) | LocalLimit(_, _) if plan.children.forall(_.isStreaming) => + throwError("Limits are not supported on streaming DataFrames/Datasets") + + case Sort(_, _, _) | SortPartitions(_, _) if plan.children.forall(_.isStreaming) => + throwError("Sorting is not supported on streaming DataFrames/Datasets") + + case Sample(_, _, _, _, child) if child.isStreaming => + throwError("Sampling is not supported on streaming DataFrames/Datasets") + + case Window(_, _, _, child) if child.isStreaming => + throwError("Non-time-based windows are not supported on streaming DataFrames/Datasets") + + case ReturnAnswer(child) if child.isStreaming => + throwError("Cannot return immediate result on streaming DataFrames/Dataset. Queries " + + "with streaming DataFrames/Datasets must be executed with write.startStream().") + + case _ => + } + } + } + + private def throwErrorIf( + condition: Boolean, + msg: String)(implicit operator: LogicalPlan): Unit = { + if (condition) { + throwError(msg) + } + } + + private def throwError(msg: String)(implicit operator: LogicalPlan): Nothing = { + throw new AnalysisException( + msg, operator.origin.line, operator.origin.startPosition, Some(operator)) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 1e7296664bb25..958966328bbe2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -297,6 +297,24 @@ package object dsl { condition: Option[Expression] = None): LogicalPlan = Join(logicalPlan, otherPlan, joinType, condition) + def cogroup[Key: Encoder, Left: Encoder, Right: Encoder, Result: Encoder]( + otherPlan: LogicalPlan, + func: (Key, Iterator[Left], Iterator[Right]) => TraversableOnce[Result], + leftGroup: Seq[Attribute], + rightGroup: Seq[Attribute], + leftAttr: Seq[Attribute], + rightAttr: Seq[Attribute] + ): LogicalPlan = { + CoGroup.apply[Key, Left, Right, Result]( + func, + leftGroup, + rightGroup, + leftAttr, + rightAttr, + logicalPlan, + otherPlan) + } + def orderBy(sortExprs: SortOrder*): LogicalPlan = Sort(sortExprs, true, logicalPlan) def sortBy(sortExprs: SortOrder*): LogicalPlan = Sort(sortExprs, false, logicalPlan) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index aceeb8aadcf68..45ac126a72f5c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -42,6 +42,9 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { */ def analyzed: Boolean = _analyzed + /** Returns true if this subtree contains any streaming data sources. */ + def isStreaming: Boolean = children.exists(_.isStreaming == true) + /** * Returns a copy of this node where `rule` has been recursively applied first to all of its * children and then itself (post-order). When `rule` does not apply to a given node, it is left diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala new file mode 100644 index 0000000000000..ce00a03e764fd --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala @@ -0,0 +1,384 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.analysis + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.types.IntegerType + +class UnsupportedOperationsSuite extends SparkFunSuite { + + val attribute = AttributeReference("a", IntegerType, nullable = true)() + val batchRelation = LocalRelation(attribute) + val streamRelation = new TestStreamingRelation(attribute) + + /* + ======================================================================================= + BATCH QUERIES + ======================================================================================= + */ + + assertSupportedInBatchPlan("local relation", batchRelation) + + assertNotSupportedInBatchPlan( + "streaming source", + streamRelation, + Seq("with streaming source", "startStream")) + + assertNotSupportedInBatchPlan( + "select on streaming source", + streamRelation.select($"count(*)"), + Seq("with streaming source", "startStream")) + + + /* + ======================================================================================= + STREAMING QUERIES + ======================================================================================= + */ + + // Batch plan in streaming query + testError( + "streaming plan - no streaming source", + Seq("without streaming source", "startStream")) { + UnsupportedOperationChecker.checkForStreaming(batchRelation.select($"count(*)"), Append) + } + + // Commands + assertNotSupportedInStreamingPlan( + "commmands", + DescribeFunction("func", true), + outputMode = Append, + expectedMsgs = "commands" :: Nil) + + // Aggregates: Not supported on streams in Append mode + assertSupportedInStreamingPlan( + "aggregate - batch with update output mode", + batchRelation.groupBy("a")("count(*)"), + outputMode = Update) + + assertSupportedInStreamingPlan( + "aggregate - batch with append output mode", + batchRelation.groupBy("a")("count(*)"), + outputMode = Append) + + assertSupportedInStreamingPlan( + "aggregate - stream with update output mode", + streamRelation.groupBy("a")("count(*)"), + outputMode = Update) + + assertNotSupportedInStreamingPlan( + "aggregate - stream with append output mode", + streamRelation.groupBy("a")("count(*)"), + outputMode = Append, + Seq("aggregation", "append output mode")) + + // Inner joins: Stream-stream not supported + testBinaryOperationInStreamingPlan( + "inner join", + _.join(_, joinType = Inner), + streamStreamSupported = false) + + // Full outer joins: only batch-batch is allowed + testBinaryOperationInStreamingPlan( + "full outer join", + _.join(_, joinType = FullOuter), + streamStreamSupported = false, + batchStreamSupported = false, + streamBatchSupported = false) + + // Left outer joins: *-stream not allowed + testBinaryOperationInStreamingPlan( + "left outer join", + _.join(_, joinType = LeftOuter), + streamStreamSupported = false, + batchStreamSupported = false, + expectedMsg = "left outer/semi/anti joins") + + // Left semi joins: stream-* not allowed + testBinaryOperationInStreamingPlan( + "left semi join", + _.join(_, joinType = LeftSemi), + streamStreamSupported = false, + batchStreamSupported = false, + expectedMsg = "left outer/semi/anti joins") + + // Left anti joins: stream-* not allowed + testBinaryOperationInStreamingPlan( + "left anti join", + _.join(_, joinType = LeftAnti), + streamStreamSupported = false, + batchStreamSupported = false, + expectedMsg = "left outer/semi/anti joins") + + // Right outer joins: stream-* not allowed + testBinaryOperationInStreamingPlan( + "right outer join", + _.join(_, joinType = RightOuter), + streamStreamSupported = false, + streamBatchSupported = false) + + // Cogroup: only batch-batch is allowed + testBinaryOperationInStreamingPlan( + "cogroup", + genCogroup, + streamStreamSupported = false, + batchStreamSupported = false, + streamBatchSupported = false) + + def genCogroup(left: LogicalPlan, right: LogicalPlan): LogicalPlan = { + def func(k: Int, left: Iterator[Int], right: Iterator[Int]): Iterator[Int] = { + Iterator.empty + } + implicit val intEncoder = ExpressionEncoder[Int] + + left.cogroup[Int, Int, Int, Int]( + right, + func, + AppendColumns[Int, Int]((x: Int) => x, left).newColumns, + AppendColumns[Int, Int]((x: Int) => x, right).newColumns, + left.output, + right.output) + } + + // Union: Mixing between stream and batch not supported + testBinaryOperationInStreamingPlan( + "union", + _.union(_), + streamBatchSupported = false, + batchStreamSupported = false) + + // Except: *-stream not supported + testBinaryOperationInStreamingPlan( + "except", + _.except(_), + streamStreamSupported = false, + batchStreamSupported = false) + + // Intersect: stream-stream not supported + testBinaryOperationInStreamingPlan( + "intersect", + _.intersect(_), + streamStreamSupported = false) + + + // Unary operations + testUnaryOperatorInStreamingPlan("sort", Sort(Nil, true, _)) + testUnaryOperatorInStreamingPlan("sort partitions", SortPartitions(Nil, _), expectedMsg = "sort") + testUnaryOperatorInStreamingPlan( + "sample", Sample(0.1, 1, true, 1L, _)(), expectedMsg = "sampling") + testUnaryOperatorInStreamingPlan( + "window", Window(Nil, Nil, Nil, _), expectedMsg = "non-time-based windows") + + + /* + ======================================================================================= + TESTING FUNCTIONS + ======================================================================================= + */ + + /** + * Test that an unary operator correctly fails support check when it has a streaming child plan, + * but not when it has batch child plan. There can be batch sub-plans inside a streaming plan, + * so it is valid for the operator to have a batch child plan. + * + * This test wraps the logical plan in a fake operator that makes the whole plan look like + * a streaming plan even if the child plan is a batch plan. This is to test that the operator + * supports having a batch child plan, forming a batch subplan inside a streaming plan. + */ + def testUnaryOperatorInStreamingPlan( + operationName: String, + logicalPlanGenerator: LogicalPlan => LogicalPlan, + outputMode: OutputMode = Append, + expectedMsg: String = ""): Unit = { + + val expectedMsgs = if (expectedMsg.isEmpty) Seq(operationName) else Seq(expectedMsg) + + assertNotSupportedInStreamingPlan( + s"$operationName with stream relation", + wrapInStreaming(logicalPlanGenerator(streamRelation)), + outputMode, + expectedMsgs) + + assertSupportedInStreamingPlan( + s"$operationName with batch relation", + wrapInStreaming(logicalPlanGenerator(batchRelation)), + outputMode) + } + + + /** + * Test that a binary operator correctly fails support check when it has combinations of + * streaming and batch child plans. There can be batch sub-plans inside a streaming plan, + * so it is valid for the operator to have a batch child plan. + */ + def testBinaryOperationInStreamingPlan( + operationName: String, + planGenerator: (LogicalPlan, LogicalPlan) => LogicalPlan, + outputMode: OutputMode = Append, + streamStreamSupported: Boolean = true, + streamBatchSupported: Boolean = true, + batchStreamSupported: Boolean = true, + expectedMsg: String = ""): Unit = { + + val expectedMsgs = if (expectedMsg.isEmpty) Seq(operationName) else Seq(expectedMsg) + + if (streamStreamSupported) { + assertSupportedInStreamingPlan( + s"$operationName with stream-stream relations", + planGenerator(streamRelation, streamRelation), + outputMode) + } else { + assertNotSupportedInStreamingPlan( + s"$operationName with stream-stream relations", + planGenerator(streamRelation, streamRelation), + outputMode, + expectedMsgs) + } + + if (streamBatchSupported) { + assertSupportedInStreamingPlan( + s"$operationName with stream-batch relations", + planGenerator(streamRelation, batchRelation), + outputMode) + } else { + assertNotSupportedInStreamingPlan( + s"$operationName with stream-batch relations", + planGenerator(streamRelation, batchRelation), + outputMode, + expectedMsgs) + } + + if (batchStreamSupported) { + assertSupportedInStreamingPlan( + s"$operationName with batch-stream relations", + planGenerator(batchRelation, streamRelation), + outputMode) + } else { + assertNotSupportedInStreamingPlan( + s"$operationName with batch-stream relations", + planGenerator(batchRelation, streamRelation), + outputMode, + expectedMsgs) + } + + assertSupportedInStreamingPlan( + s"$operationName with batch-batch relations", + planGenerator(batchRelation, batchRelation), + outputMode) + } + + /** + * Assert that the logical plan is supported as subplan insider a streaming plan. + * + * To test this correctly, the given logical plan is wrapped in a fake operator that makes the + * whole plan look like a streaming plan. Otherwise, a batch plan may throw not supported + * exception simply for not being a streaming plan, even though that plan could exists as batch + * subplan inside some streaming plan. + */ + def assertSupportedInStreamingPlan( + name: String, + plan: LogicalPlan, + outputMode: OutputMode): Unit = { + test(s"streaming plan - $name: supported") { + UnsupportedOperationChecker.checkForStreaming(wrapInStreaming(plan), outputMode) + } + } + + /** + * Assert that the logical plan is not supported inside a streaming plan. + * + * To test this correctly, the given logical plan is wrapped in a fake operator that makes the + * whole plan look like a streaming plan. Otherwise, a batch plan may throw not supported + * exception simply for not being a streaming plan, even though that plan could exists as batch + * subplan inside some streaming plan. + */ + def assertNotSupportedInStreamingPlan( + name: String, + plan: LogicalPlan, + outputMode: OutputMode, + expectedMsgs: Seq[String]): Unit = { + testError( + s"streaming plan - $name: not supported", + expectedMsgs :+ "streaming" :+ "DataFrame" :+ "Dataset" :+ "not supported") { + UnsupportedOperationChecker.checkForStreaming(wrapInStreaming(plan), outputMode) + } + } + + /** Assert that the logical plan is supported as a batch plan */ + def assertSupportedInBatchPlan(name: String, plan: LogicalPlan): Unit = { + test(s"batch plan - $name: supported") { + UnsupportedOperationChecker.checkForBatch(plan) + } + } + + /** Assert that the logical plan is not supported as a batch plan */ + def assertNotSupportedInBatchPlan( + name: String, + plan: LogicalPlan, + expectedMsgs: Seq[String]): Unit = { + testError(s"batch plan - $name: not supported", expectedMsgs) { + UnsupportedOperationChecker.checkForBatch(plan) + } + } + + /** + * Test whether the body of code will fail. If it does fail, then check if it has expected + * messages. + */ + def testError(testName: String, expectedMsgs: Seq[String])(testBody: => Unit): Unit = { + + test(testName) { + val e = intercept[AnalysisException] { + testBody + } + + if (!expectedMsgs.map(_.toLowerCase).forall(e.getMessage.toLowerCase.contains)) { + fail( + s"""Exception message should contain the following substrings: + | + | ${expectedMsgs.mkString("\n ")} + | + |Actual exception message: + | + | ${e.getMessage} + """.stripMargin) + } + } + } + + def wrapInStreaming(plan: LogicalPlan): LogicalPlan = { + new StreamingPlanWrapper(plan) + } + + case class StreamingPlanWrapper(child: LogicalPlan) extends UnaryNode { + override def output: Seq[Attribute] = child.output + override def isStreaming: Boolean = true + } + + case class TestStreamingRelation(output: Seq[Attribute]) extends LeafNode { + def this(attribute: Attribute) = this(Seq(attribute)) + override def isStreaming: Boolean = true + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala index faef9ed274593..cc86f1f6e2f48 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala @@ -18,7 +18,9 @@ package org.apache.spark.sql.catalyst.plans import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.types.IntegerType /** * This suite is used to test [[LogicalPlan]]'s `resolveOperators` and make sure it can correctly @@ -68,4 +70,23 @@ class LogicalPlanSuite extends SparkFunSuite { assert(invocationCount === 1) } + + test("isStreaming") { + val relation = LocalRelation(AttributeReference("a", IntegerType, nullable = true)()) + val incrementalRelation = new LocalRelation( + Seq(AttributeReference("a", IntegerType, nullable = true)())) { + override def isStreaming(): Boolean = true + } + + case class TestBinaryRelation(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { + override def output: Seq[Attribute] = left.output ++ right.output + } + + require(relation.isStreaming === false) + require(incrementalRelation.isStreaming === true) + assert(TestBinaryRelation(relation, relation).isStreaming === false) + assert(TestBinaryRelation(incrementalRelation, relation).isStreaming === true) + assert(TestBinaryRelation(relation, incrementalRelation).isStreaming === true) + assert(TestBinaryRelation(incrementalRelation, incrementalRelation).isStreaming) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala index 1343e81569cbd..39d04ed8c24f8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala @@ -20,8 +20,10 @@ package org.apache.spark.sql import scala.collection.mutable import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.catalyst.analysis.{Append, OutputMode, UnsupportedOperationChecker} import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.util.ContinuousQueryListener /** @@ -172,14 +174,23 @@ class ContinuousQueryManager(sqlContext: SQLContext) { checkpointLocation: String, df: DataFrame, sink: Sink, - trigger: Trigger = ProcessingTime(0)): ContinuousQuery = { + trigger: Trigger = ProcessingTime(0), + outputMode: OutputMode = Append): ContinuousQuery = { activeQueriesLock.synchronized { if (activeQueries.contains(name)) { throw new IllegalArgumentException( s"Cannot start query with name $name as a query with that name is already active") } + val analyzedPlan = df.queryExecution.analyzed + df.queryExecution.assertAnalyzed() + + if (sqlContext.conf.getConf(SQLConf.UNSUPPORTED_OPERATION_CHECK_ENABLED)) { + UnsupportedOperationChecker.checkForStreaming(analyzedPlan, outputMode) + } + var nextSourceId = 0L - val logicalPlan = df.logicalPlan.transform { + + val logicalPlan = analyzedPlan.transform { case StreamingRelation(dataSource, _, output) => // Materialize source to avoid creating it in every batch val metadataPath = s"$checkpointLocation/sources/$nextSourceId" @@ -195,6 +206,7 @@ class ContinuousQueryManager(sqlContext: SQLContext) { checkpointLocation, logicalPlan, sink, + outputMode, trigger) query.start() activeQueries.put(name, query) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index fb3e184a640da..1a09d70fb94c3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -461,9 +461,7 @@ class Dataset[T] private[sql]( * @since 2.0.0 */ @Experimental - def isStreaming: Boolean = logicalPlan.find { n => - n.isInstanceOf[StreamingRelation] || n.isInstanceOf[StreamingExecutionRelation] - }.isDefined + def isStreaming: Boolean = logicalPlan.isStreaming /** * Displays the [[Dataset]] in a tabular form. Strings more than 20 characters will be truncated, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index f5e1e77263b5b..ddcae0fe07533 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -20,9 +20,11 @@ package org.apache.spark.sql.execution import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, SQLContext} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange} +import org.apache.spark.sql.internal.SQLConf /** * The primary workflow for executing relational queries using Spark. Designed to allow easy @@ -43,10 +45,17 @@ class QueryExecution(val sqlContext: SQLContext, val logical: LogicalPlan) { throw ae } + def assertSupported(): Unit = { + if (sqlContext.conf.getConf(SQLConf.UNSUPPORTED_OPERATION_CHECK_ENABLED)) { + UnsupportedOperationChecker.checkForBatch(analyzed) + } + } + lazy val analyzed: LogicalPlan = sqlContext.sessionState.analyzer.execute(logical) lazy val withCachedData: LogicalPlan = { assertAnalyzed() + assertSupported() sqlContext.cacheManager.useCachedData(analyzed) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index aaced49dd16ce..81244ed874498 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -18,9 +18,11 @@ package org.apache.spark.sql.execution.streaming import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.analysis.{OutputMode, UnsupportedOperationChecker} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{QueryExecution, SparkPlan, SparkPlanner, UnaryNode} +import org.apache.spark.sql.internal.SQLConf /** * A variant of [[QueryExecution]] that allows the execution of the given [[LogicalPlan]] @@ -29,6 +31,7 @@ import org.apache.spark.sql.execution.{QueryExecution, SparkPlan, SparkPlanner, class IncrementalExecution( ctx: SQLContext, logicalPlan: LogicalPlan, + outputMode: OutputMode, checkpointLocation: String, currentBatchId: Long) extends QueryExecution(ctx, logicalPlan) { @@ -69,4 +72,7 @@ class IncrementalExecution( } override def preparations: Seq[Rule[SparkPlan]] = state +: super.preparations + + /** No need assert supported, as this check has already been done */ + override def assertSupported(): Unit = { } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 87dd27a2b1aed..2a1fa1ba627c8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.analysis.OutputMode import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} @@ -48,6 +49,7 @@ class StreamExecution( checkpointRoot: String, private[sql] val logicalPlan: LogicalPlan, val sink: Sink, + val outputMode: OutputMode, val trigger: Trigger) extends ContinuousQuery with Logging { /** An monitor used to wait/notify when batches complete. */ @@ -314,8 +316,13 @@ class StreamExecution( } val optimizerStart = System.nanoTime() - lastExecution = - new IncrementalExecution(sqlContext, newPlan, checkpointFile("state"), currentBatchId) + lastExecution = new IncrementalExecution( + sqlContext, + newPlan, + outputMode, + checkpointFile("state"), + currentBatchId) + lastExecution.executedPlan val optimizerTime = (System.nanoTime() - optimizerStart).toDouble / 1000000 logDebug(s"Optimized batch in ${optimizerTime}ms") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala index d2872e49ce28a..c29291eb584a0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala @@ -37,6 +37,7 @@ object StreamingRelation { */ case class StreamingRelation(dataSource: DataSource, sourceName: String, output: Seq[Attribute]) extends LeafNode { + override def isStreaming: Boolean = true override def toString: String = sourceName } @@ -45,6 +46,7 @@ case class StreamingRelation(dataSource: DataSource, sourceName: String, output: * [[org.apache.spark.sql.catalyst.plans.logical.LogicalPlan]]. */ case class StreamingExecutionRelation(source: Source, output: Seq[Attribute]) extends LeafNode { + override def isStreaming: Boolean = true override def toString: String = source.toString } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 70e18cebdd7fb..7f206bdb9b240 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -442,6 +442,14 @@ object SQLConf { .stringConf .createOptional + val UNSUPPORTED_OPERATION_CHECK_ENABLED = + SQLConfigBuilder("spark.sql.streaming.unsupportedOperationCheck") + .internal() + .doc("When true, the logical plan for continuous query will be checked for unsupported" + + " operations.") + .booleanConf + .createWithDefault(true) + // TODO: This is still WIP and shouldn't be turned on without extensive test coverage val COLUMNAR_AGGREGATE_MAP_ENABLED = SQLConfigBuilder("spark.sql.codegen.aggregate.map.enabled") .internal() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala index 6ccc99fe179d7..242ea9cb27361 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala @@ -33,6 +33,7 @@ import org.scalatest.exceptions.TestFailedDueToTimeoutException import org.scalatest.time.Span import org.scalatest.time.SpanSugar._ +import org.apache.spark.sql.catalyst.analysis.{Append, OutputMode} import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder, RowEncoder} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ @@ -75,6 +76,8 @@ trait StreamTest extends QueryTest with Timeouts { /** How long to wait for an active stream to catch up when checking a result. */ val streamingTimeout = 10.seconds + val outputMode: OutputMode = Append + /** A trait for actions that can be performed while testing a streaming DataFrame. */ trait StreamAction @@ -228,6 +231,7 @@ trait StreamTest extends QueryTest with Timeouts { |$testActions | |== Stream == + |Output Mode: $outputMode |Stream state: $currentOffsets |Thread state: $threadState |${if (streamDeathCause != null) stackTraceToString(streamDeathCause) else ""} @@ -235,6 +239,7 @@ trait StreamTest extends QueryTest with Timeouts { |== Sink == |${sink.toDebugString} | + | |== Plan == |${if (currentStream != null) currentStream.lastExecution else ""} """.stripMargin @@ -293,7 +298,8 @@ trait StreamTest extends QueryTest with Timeouts { StreamExecution.nextName, metadataRoot, stream, - sink) + sink, + outputMode = outputMode) .asInstanceOf[StreamExecution] currentStream.microBatchThread.setUncaughtExceptionHandler( new UncaughtExceptionHandler { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index 2bd27c7efdbdc..6f3149dbc5033 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -17,10 +17,9 @@ package org.apache.spark.sql.streaming -import org.scalatest.concurrent.Eventually._ - -import org.apache.spark.sql.{DataFrame, Row, SQLContext, StreamTest} +import org.apache.spark.sql._ import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.functions._ import org.apache.spark.sql.sources.StreamSourceProvider import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{IntegerType, StructField, StructType} @@ -108,6 +107,35 @@ class StreamSuite extends StreamTest with SharedSQLContext { assertDF(df) assertDF(df) } + + test("unsupported queries") { + val streamInput = MemoryStream[Int] + val batchInput = Seq(1, 2, 3).toDS() + + def assertError(expectedMsgs: Seq[String])(body: => Unit): Unit = { + val e = intercept[AnalysisException] { + body + } + expectedMsgs.foreach { s => assert(e.getMessage.contains(s)) } + } + + // Running streaming plan as a batch query + assertError("startStream" :: Nil) { + streamInput.toDS.map { i => i }.count() + } + + // Running non-streaming plan with as a streaming query + assertError("without streaming sources" :: "startStream" :: Nil) { + val ds = batchInput.map { i => i } + testStream(ds)() + } + + // Running streaming plan that cannot be incrementalized + assertError("not supported" :: "streaming" :: Nil) { + val ds = streamInput.toDS.map { i => i }.sort() + testStream(ds)() + } + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index 3af7c01e525ad..fa3b122f6d2da 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.streaming import org.apache.spark.SparkException import org.apache.spark.sql.StreamTest +import org.apache.spark.sql.catalyst.analysis.Update import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.expressions.scala.typed import org.apache.spark.sql.functions._ @@ -32,6 +33,8 @@ class StreamingAggregationSuite extends StreamTest with SharedSQLContext { import testImplicits._ + override val outputMode = Update + test("simple count") { val inputData = MemoryStream[Int] From b64482f49f6b9c7ff0ba64bd3202fe9cc6ad119a Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Mon, 18 Apr 2016 11:52:29 -0700 Subject: [PATCH 04/69] [SPARK-14306][ML][PYSPARK] PySpark ml.classification OneVsRest support export/import ## What changes were proposed in this pull request? https://issues.apache.org/jira/browse/SPARK-14306 Add PySpark OneVsRest save/load supports. ## How was this patch tested? Test with Python unit test. Author: Xusen Yin Closes #12439 from yinxusen/SPARK-14306-0415. --- .../spark/ml/classification/OneVsRest.scala | 7 + python/pyspark/ml/classification.py | 142 +++++++++++++++--- python/pyspark/ml/tests.py | 25 ++- 3 files changed, 151 insertions(+), 23 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala index 4de1b877b0194..f10c60a78df1a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala @@ -17,8 +17,10 @@ package org.apache.spark.ml.classification +import java.util.{List => JList} import java.util.UUID +import scala.collection.JavaConverters._ import scala.language.existentials import org.apache.hadoop.fs.Path @@ -135,6 +137,11 @@ final class OneVsRestModel private[ml] ( @Since("1.4.0") val models: Array[_ <: ClassificationModel[_, _]]) extends Model[OneVsRestModel] with OneVsRestParams with MLWritable { + /** A Python-friendly auxiliary constructor. */ + private[ml] def this(uid: String, models: JList[_ <: ClassificationModel[_, _]]) = { + this(uid, Metadata.empty, models.asScala.toArray) + } + @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema, fitting = false, getClassifier.featuresDataType) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 089316729ca7b..de1321b139751 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -23,7 +23,7 @@ from pyspark.ml.regression import ( RandomForestParams, TreeEnsembleParams, DecisionTreeModel, TreeEnsembleModels) from pyspark.ml.util import * -from pyspark.ml.wrapper import JavaEstimator, JavaModel +from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaParams from pyspark.ml.wrapper import JavaWrapper from pyspark.mllib.common import inherit_doc from pyspark.sql import DataFrame @@ -1160,8 +1160,33 @@ def weights(self): return self._call_java("weights") +class OneVsRestParams(HasFeaturesCol, HasLabelCol, HasPredictionCol): + """ + Parameters for OneVsRest and OneVsRestModel. + """ + + classifier = Param(Params._dummy(), "classifier", "base binary classifier") + + @since("2.0.0") + def setClassifier(self, value): + """ + Sets the value of :py:attr:`classifier`. + + .. note:: Only LogisticRegression and NaiveBayes are supported now. + """ + self._set(classifier=value) + return self + + @since("2.0.0") + def getClassifier(self): + """ + Gets the value of classifier or its default value. + """ + return self.getOrDefault(self.classifier) + + @inherit_doc -class OneVsRest(Estimator, HasFeaturesCol, HasLabelCol, HasPredictionCol): +class OneVsRest(Estimator, OneVsRestParams, MLReadable, MLWritable): """ Reduction of Multiclass Classification to Binary Classification. Performs reduction using one against all strategy. @@ -1195,8 +1220,6 @@ class OneVsRest(Estimator, HasFeaturesCol, HasLabelCol, HasPredictionCol): .. versionadded:: 2.0.0 """ - classifier = Param(Params._dummy(), "classifier", "base binary classifier") - @keyword_only def __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", classifier=None): @@ -1218,23 +1241,6 @@ def setParams(self, featuresCol=None, labelCol=None, predictionCol=None, classif kwargs = self.setParams._input_kwargs return self._set(**kwargs) - @since("2.0.0") - def setClassifier(self, value): - """ - Sets the value of :py:attr:`classifier`. - - .. note:: Only LogisticRegression and NaiveBayes are supported now. - """ - self._set(classifier=value) - return self - - @since("2.0.0") - def getClassifier(self): - """ - Gets the value of classifier or its default value. - """ - return self.getOrDefault(self.classifier) - def _fit(self, dataset): labelCol = self.getLabelCol() featuresCol = self.getFeaturesCol() @@ -1288,8 +1294,53 @@ def copy(self, extra=None): newOvr.setClassifier(self.getClassifier().copy(extra)) return newOvr + @since("2.0.0") + def write(self): + """Returns an MLWriter instance for this ML instance.""" + return JavaMLWriter(self) + + @since("2.0.0") + def save(self, path): + """Save this ML instance to the given path, a shortcut of `write().save(path)`.""" + self.write().save(path) + + @classmethod + @since("2.0.0") + def read(cls): + """Returns an MLReader instance for this class.""" + return JavaMLReader(cls) + + @classmethod + def _from_java(cls, java_stage): + """ + Given a Java OneVsRest, create and return a Python wrapper of it. + Used for ML persistence. + """ + featuresCol = java_stage.getFeaturesCol() + labelCol = java_stage.getLabelCol() + predictionCol = java_stage.getPredictionCol() + classifier = JavaParams._from_java(java_stage.getClassifier()) + py_stage = cls(featuresCol=featuresCol, labelCol=labelCol, predictionCol=predictionCol, + classifier=classifier) + py_stage._resetUid(java_stage.uid()) + return py_stage + + def _to_java(self): + """ + Transfer this instance to a Java OneVsRest. Used for ML persistence. + + :return: Java object equivalent to this instance. + """ + _java_obj = JavaParams._new_java_obj("org.apache.spark.ml.classification.OneVsRest", + self.uid) + _java_obj.setClassifier(self.getClassifier()._to_java()) + _java_obj.setFeaturesCol(self.getFeaturesCol()) + _java_obj.setLabelCol(self.getLabelCol()) + _java_obj.setPredictionCol(self.getPredictionCol()) + return _java_obj -class OneVsRestModel(Model, HasFeaturesCol, HasLabelCol, HasPredictionCol): + +class OneVsRestModel(Model, OneVsRestParams, MLReadable, MLWritable): """ Model fitted by OneVsRest. This stores the models resulting from training k binary classifiers: one for each class. @@ -1367,6 +1418,53 @@ def copy(self, extra=None): newModel.models = [model.copy(extra) for model in self.models] return newModel + @since("2.0.0") + def write(self): + """Returns an MLWriter instance for this ML instance.""" + return JavaMLWriter(self) + + @since("2.0.0") + def save(self, path): + """Save this ML instance to the given path, a shortcut of `write().save(path)`.""" + self.write().save(path) + + @classmethod + @since("2.0.0") + def read(cls): + """Returns an MLReader instance for this class.""" + return JavaMLReader(cls) + + @classmethod + def _from_java(cls, java_stage): + """ + Given a Java OneVsRestModel, create and return a Python wrapper of it. + Used for ML persistence. + """ + featuresCol = java_stage.getFeaturesCol() + labelCol = java_stage.getLabelCol() + predictionCol = java_stage.getPredictionCol() + classifier = JavaParams._from_java(java_stage.getClassifier()) + models = [JavaParams._from_java(model) for model in java_stage.models()] + py_stage = cls(models=models).setPredictionCol(predictionCol).setLabelCol(labelCol)\ + .setFeaturesCol(featuresCol).setClassifier(classifier) + py_stage._resetUid(java_stage.uid()) + return py_stage + + def _to_java(self): + """ + Transfer this instance to a Java OneVsRestModel. Used for ML persistence. + + :return: Java object equivalent to this instance. + """ + java_models = [model._to_java() for model in self.models] + _java_obj = JavaParams._new_java_obj("org.apache.spark.ml.classification.OneVsRestModel", + self.uid, java_models) + _java_obj.set("classifier", self.getClassifier()._to_java()) + _java_obj.set("featuresCol", self.getFeaturesCol()) + _java_obj.set("labelCol", self.getLabelCol()) + _java_obj.set("predictionCol", self.getPredictionCol()) + return _java_obj + if __name__ == "__main__": import doctest diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index a7a9868baccb3..9d6ff47b54541 100644 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -43,7 +43,8 @@ import numpy as np from pyspark.ml import Estimator, Model, Pipeline, PipelineModel, Transformer -from pyspark.ml.classification import LogisticRegression, DecisionTreeClassifier, OneVsRest +from pyspark.ml.classification import ( + LogisticRegression, DecisionTreeClassifier, OneVsRest, OneVsRestModel) from pyspark.ml.clustering import KMeans from pyspark.ml.evaluation import BinaryClassificationEvaluator, RegressionEvaluator from pyspark.ml.feature import * @@ -881,6 +882,28 @@ def test_output_columns(self): output = model.transform(df) self.assertEqual(output.columns, ["label", "features", "prediction"]) + def test_save_load(self): + temp_path = tempfile.mkdtemp() + sqlContext = SQLContext(self.sc) + df = sqlContext.createDataFrame([(0.0, Vectors.dense(1.0, 0.8)), + (1.0, Vectors.sparse(2, [], [])), + (2.0, Vectors.dense(0.5, 0.5))], + ["label", "features"]) + lr = LogisticRegression(maxIter=5, regParam=0.01) + ovr = OneVsRest(classifier=lr) + model = ovr.fit(df) + ovrPath = temp_path + "/ovr" + ovr.save(ovrPath) + loadedOvr = OneVsRest.load(ovrPath) + self.assertEqual(loadedOvr.getFeaturesCol(), ovr.getFeaturesCol()) + self.assertEqual(loadedOvr.getLabelCol(), ovr.getLabelCol()) + self.assertEqual(loadedOvr.getClassifier().uid, ovr.getClassifier().uid) + modelPath = temp_path + "/ovrModel" + model.save(modelPath) + loadedModel = OneVsRestModel.load(modelPath) + for m, n in zip(model.models, loadedModel.models): + self.assertEqual(m.uid, n.uid) + class HashingTFTest(PySparkTestCase): From d280d1da1aec925687a0bfb496f3a6e0979e896f Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 18 Apr 2016 12:26:56 -0700 Subject: [PATCH 05/69] [SPARK-14580][SPARK-14655][SQL] Hive IfCoercion should preserve predicate. ## What changes were proposed in this pull request? Currently, `HiveTypeCoercion.IfCoercion` removes all predicates whose return-type are null. However, some UDFs need evaluations because they are designed to throw exceptions. This PR fixes that to preserve the predicates. Also, `assert_true` is implemented as Spark SQL function. **Before** ``` scala> sql("select if(assert_true(false),2,3)").head res2: org.apache.spark.sql.Row = [3] ``` **After** ``` scala> sql("select if(assert_true(false),2,3)").head ... ASSERT_TRUE ... ``` **Hive** ``` hive> select if(assert_true(false),2,3); OK Failed with exception java.io.IOException:org.apache.hadoop.hive.ql.metadata.HiveException: ASSERT_TRUE(): assertion failed. ``` ## How was this patch tested? Pass the Jenkins tests (including a new testcase in `HivePlanTest`) Author: Dongjoon Hyun Closes #12340 from dongjoon-hyun/SPARK-14580. --- .../catalyst/analysis/FunctionRegistry.scala | 1 + .../catalyst/analysis/HiveTypeCoercion.scala | 6 +-- .../spark/sql/catalyst/expressions/misc.scala | 38 +++++++++++++++++++ .../analysis/HiveTypeCoercionSuite.scala | 15 ++++++-- .../expressions/MiscFunctionsSuite.scala | 17 +++++++++ 5 files changed, 70 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index ed19191b72f0e..a44430059dddd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -329,6 +329,7 @@ object FunctionRegistry { expression[SortArray]("sort_array"), // misc functions + expression[AssertTrue]("assert_true"), expression[Crc32]("crc32"), expression[Md5]("md5"), expression[Murmur3Hash]("hash"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 823d2495fad80..5323b79c57c4b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -584,10 +584,10 @@ object HiveTypeCoercion { val newRight = if (right.dataType == widestType) right else Cast(right, widestType) If(pred, newLeft, newRight) }.getOrElse(i) // If there is no applicable conversion, leave expression unchanged. - // Convert If(null literal, _, _) into boolean type. - // In the optimizer, we should short-circuit this directly into false value. - case If(pred, left, right) if pred.dataType == NullType => + case If(Literal(null, NullType), left, right) => If(Literal.create(null, BooleanType), left, right) + case If(pred, left, right) if pred.dataType == NullType => + If(Cast(pred, BooleanType), left, right) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index 113fc862c7ded..f2f0c2d698efa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -486,6 +486,44 @@ case class PrintToStderr(child: Expression) extends UnaryExpression { } } +/** + * A function throws an exception if 'condition' is not true. + */ +@ExpressionDescription( + usage = "_FUNC_(condition) - Throw an exception if 'condition' is not true.") +case class AssertTrue(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { + + override def nullable: Boolean = true + + override def inputTypes: Seq[DataType] = Seq(BooleanType) + + override def dataType: DataType = NullType + + override def prettyName: String = "assert_true" + + override def eval(input: InternalRow) : Any = { + val v = child.eval(input) + if (v == null || java.lang.Boolean.FALSE.equals(v)) { + throw new RuntimeException(s"'${child.simpleString}' is not true!") + } else { + null + } + } + + override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval = child.gen(ctx) + ev.isNull = "true" + ev.value = "null" + s"""${eval.code} + |if (${eval.isNull} || !${eval.value}) { + | throw new RuntimeException("'${child.simpleString}' is not true."); + |} + """.stripMargin + } + + override def sql: String = s"assert_true(${child.sql})" +} + /** * A xxHash64 64-bit hash expression. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala index 883ef48984d79..18de8b152b070 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -348,15 +348,22 @@ class HiveTypeCoercionSuite extends PlanTest { test("type coercion for If") { val rule = HiveTypeCoercion.IfCoercion + ruleTest(rule, If(Literal(true), Literal(1), Literal(1L)), - If(Literal(true), Cast(Literal(1), LongType), Literal(1L)) - ) + If(Literal(true), Cast(Literal(1), LongType), Literal(1L))) ruleTest(rule, If(Literal.create(null, NullType), Literal(1), Literal(1)), - If(Literal.create(null, BooleanType), Literal(1), Literal(1)) - ) + If(Literal.create(null, BooleanType), Literal(1), Literal(1))) + + ruleTest(rule, + If(AssertTrue(Literal.create(true, BooleanType)), Literal(1), Literal(2)), + If(Cast(AssertTrue(Literal.create(true, BooleanType)), BooleanType), Literal(1), Literal(2))) + + ruleTest(rule, + If(AssertTrue(Literal.create(false, BooleanType)), Literal(1), Literal(2)), + If(Cast(AssertTrue(Literal.create(false, BooleanType)), BooleanType), Literal(1), Literal(2))) } test("type coercion for CaseKeyWhen") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MiscFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MiscFunctionsSuite.scala index f5bafcc6a783e..56de82237bd71 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MiscFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MiscFunctionsSuite.scala @@ -69,6 +69,23 @@ class MiscFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkConsistencyBetweenInterpretedAndCodegen(Crc32, BinaryType) } + test("assert_true") { + intercept[RuntimeException] { + checkEvaluation(AssertTrue(Literal(false, BooleanType)), null) + } + intercept[RuntimeException] { + checkEvaluation(AssertTrue(Cast(Literal(0), BooleanType)), null) + } + intercept[RuntimeException] { + checkEvaluation(AssertTrue(Literal.create(null, NullType)), null) + } + intercept[RuntimeException] { + checkEvaluation(AssertTrue(Literal.create(null, BooleanType)), null) + } + checkEvaluation(AssertTrue(Literal(true, BooleanType)), null) + checkEvaluation(AssertTrue(Cast(Literal(1), BooleanType)), null) + } + private val structOfString = new StructType().add("str", StringType) private val structOfUDT = new StructType().add("udt", new ExamplePointUDT, false) private val arrayOfString = ArrayType(StringType) From 3d66a2ce9bfc19096e07181f9e970372d32bbc0b Mon Sep 17 00:00:00 2001 From: Jason Lee Date: Mon, 18 Apr 2016 12:47:14 -0700 Subject: [PATCH 06/69] [SPARK-14564][ML][MLLIB][PYSPARK] Python Word2Vec missing setWindowSize method ## What changes were proposed in this pull request? Added windowSize getter/setter to ML/MLlib ## How was this patch tested? Added test cases in tests.py under both ML and MLlib Author: Jason Lee Closes #12428 from jasoncl/SPARK-14564. --- .../mllib/api/python/PythonMLLibAPI.scala | 5 +++- python/pyspark/ml/feature.py | 28 +++++++++++++++---- python/pyspark/ml/tests.py | 5 ++++ python/pyspark/mllib/feature.py | 11 +++++++- python/pyspark/mllib/tests.py | 4 ++- 5 files changed, 45 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 1a58779055f44..32dc16de08469 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -671,6 +671,7 @@ private[python] class PythonMLLibAPI extends Serializable { * @param numPartitions number of partitions * @param numIterations number of iterations * @param seed initial seed for random generator + * @param windowSize size of window * @return A handle to java Word2VecModelWrapper instance at python side */ def trainWord2VecModel( @@ -680,7 +681,8 @@ private[python] class PythonMLLibAPI extends Serializable { numPartitions: Int, numIterations: Int, seed: Long, - minCount: Int): Word2VecModelWrapper = { + minCount: Int, + windowSize: Int): Word2VecModelWrapper = { val word2vec = new Word2Vec() .setVectorSize(vectorSize) .setLearningRate(learningRate) @@ -688,6 +690,7 @@ private[python] class PythonMLLibAPI extends Serializable { .setNumIterations(numIterations) .setSeed(seed) .setMinCount(minCount) + .setWindowSize(windowSize) try { val model = word2vec.fit(dataJRDD.rdd.persist(StorageLevel.MEMORY_AND_DISK_SER)) new Word2VecModelWrapper(model) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 776906eaab79b..49a78ede37d17 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -2219,28 +2219,31 @@ class Word2Vec(JavaEstimator, HasStepSize, HasMaxIter, HasSeed, HasInputCol, Has minCount = Param(Params._dummy(), "minCount", "the minimum number of times a token must appear to be included in the " + "word2vec model's vocabulary", typeConverter=TypeConverters.toInt) + windowSize = Param(Params._dummy(), "windowSize", + "the window size (context words from [-window, window]). Default value is 5", + typeConverter=TypeConverters.toInt) @keyword_only def __init__(self, vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, - seed=None, inputCol=None, outputCol=None): + seed=None, inputCol=None, outputCol=None, windowSize=5): """ __init__(self, vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, \ - seed=None, inputCol=None, outputCol=None) + seed=None, inputCol=None, outputCol=None, windowSize=5) """ super(Word2Vec, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.Word2Vec", self.uid) self._setDefault(vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, - seed=None) + seed=None, windowSize=5) kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @keyword_only @since("1.4.0") def setParams(self, vectorSize=100, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, - seed=None, inputCol=None, outputCol=None): + seed=None, inputCol=None, outputCol=None, windowSize=5): """ setParams(self, minCount=5, numPartitions=1, stepSize=0.025, maxIter=1, seed=None, \ - inputCol=None, outputCol=None) + inputCol=None, outputCol=None, windowSize=5) Sets params for this Word2Vec. """ kwargs = self.setParams._input_kwargs @@ -2291,6 +2294,21 @@ def getMinCount(self): """ return self.getOrDefault(self.minCount) + @since("2.0.0") + def setWindowSize(self, value): + """ + Sets the value of :py:attr:`windowSize`. + """ + self._set(windowSize=value) + return self + + @since("2.0.0") + def getWindowSize(self): + """ + Gets the value of windowSize or its default value. + """ + return self.getOrDefault(self.windowSize) + def _create_model(self, java_model): return Word2VecModel(java_model) diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index 9d6ff47b54541..f1bca6ebe0ce7 100644 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -341,6 +341,11 @@ def test_param_property_error(self): params = param_store.params # should not invoke the property 'test_property' self.assertEqual(len(params), 1) + def test_word2vec_param(self): + model = Word2Vec().setWindowSize(6) + # Check windowSize is set properly + self.assertEqual(model.getWindowSize(), 6) + class FeatureTests(PySparkTestCase): diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index b3dd2f63a5d80..90559f6cfbe43 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -617,6 +617,7 @@ def __init__(self): self.numIterations = 1 self.seed = random.randint(0, sys.maxsize) self.minCount = 5 + self.windowSize = 5 @since('1.2.0') def setVectorSize(self, vectorSize): @@ -669,6 +670,14 @@ def setMinCount(self, minCount): self.minCount = minCount return self + @since('2.0.0') + def setWindowSize(self, windowSize): + """ + Sets window size (default: 5). + """ + self.windowSize = windowSize + return self + @since('1.2.0') def fit(self, data): """ @@ -682,7 +691,7 @@ def fit(self, data): jmodel = callMLlibFunc("trainWord2VecModel", data, int(self.vectorSize), float(self.learningRate), int(self.numPartitions), int(self.numIterations), int(self.seed), - int(self.minCount)) + int(self.minCount), int(self.windowSize)) return Word2VecModel(jmodel) diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index ac55fbf79841f..f272da56d1aee 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -1027,13 +1027,15 @@ def test_word2vec_setters(self): .setNumPartitions(2) \ .setNumIterations(10) \ .setSeed(1024) \ - .setMinCount(3) + .setMinCount(3) \ + .setWindowSize(6) self.assertEqual(model.vectorSize, 2) self.assertTrue(model.learningRate < 0.02) self.assertEqual(model.numPartitions, 2) self.assertEqual(model.numIterations, 10) self.assertEqual(model.seed, 1024) self.assertEqual(model.minCount, 3) + self.assertEqual(model.windowSize, 6) def test_word2vec_get_vectors(self): data = [ From e4ae974294fc61f03b235f82d1618f29cad8feee Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 18 Apr 2016 12:57:23 -0700 Subject: [PATCH 07/69] [HOTFIX] Fix Scala 2.10 compilation break. --- .../spark/sql/catalyst/expressions/MiscFunctionsSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MiscFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MiscFunctionsSuite.scala index 56de82237bd71..33916c0891866 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MiscFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MiscFunctionsSuite.scala @@ -71,7 +71,7 @@ class MiscFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("assert_true") { intercept[RuntimeException] { - checkEvaluation(AssertTrue(Literal(false, BooleanType)), null) + checkEvaluation(AssertTrue(Literal.create(false, BooleanType)), null) } intercept[RuntimeException] { checkEvaluation(AssertTrue(Cast(Literal(0), BooleanType)), null) @@ -82,7 +82,7 @@ class MiscFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { intercept[RuntimeException] { checkEvaluation(AssertTrue(Literal.create(null, BooleanType)), null) } - checkEvaluation(AssertTrue(Literal(true, BooleanType)), null) + checkEvaluation(AssertTrue(Literal.create(true, BooleanType)), null) checkEvaluation(AssertTrue(Cast(Literal(1), BooleanType)), null) } From 28ee15702d9efd52a26a065c6e544b5345a8f65d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 18 Apr 2016 13:15:23 -0700 Subject: [PATCH 08/69] [SPARK-14647][SQL] Group SQLContext/HiveContext state into SharedState ## What changes were proposed in this pull request? This patch adds a SharedState that groups state shared across multiple SQLContexts. This is analogous to the SessionState added in SPARK-13526 that groups session-specific state. This cleanup makes the constructors of the contexts simpler and ultimately allows us to remove HiveContext in the near future. ## How was this patch tested? Existing tests. Author: Yin Huai Closes #12463 from yhuai/sharedState. --- .../org/apache/spark/sql/SQLContext.scala | 31 +++---- .../spark/sql/internal/SessionState.scala | 2 - .../spark/sql/internal/SharedState.scala | 47 ++++++++++ .../apache/spark/sql/hive/HiveContext.scala | 51 ++++------- .../spark/sql/hive/HiveSessionState.scala | 15 +++- .../spark/sql/hive/HiveSharedState.scala | 53 ++++++++++++ .../apache/spark/sql/hive/test/TestHive.scala | 86 +++++++------------ .../sql/hive/HiveExternalCatalogSuite.scala | 12 +-- 8 files changed, 175 insertions(+), 122 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 9259ff40625c9..781d6998190b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -41,7 +41,7 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.ShowTablesCommand import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.ui.{SQLListener, SQLTab} -import org.apache.spark.sql.internal.{SessionState, SQLConf} +import org.apache.spark.sql.internal.{SessionState, SharedState, SQLConf} import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types._ import org.apache.spark.sql.util.ExecutionListenerManager @@ -63,17 +63,14 @@ import org.apache.spark.util.Utils * @since 1.0.0 */ class SQLContext private[sql]( - @transient val sparkContext: SparkContext, - @transient protected[sql] val cacheManager: CacheManager, - @transient private[sql] val listener: SQLListener, - val isRootContext: Boolean, - @transient private[sql] val externalCatalog: ExternalCatalog) + @transient protected[sql] val sharedState: SharedState, + val isRootContext: Boolean) extends Logging with Serializable { self => def this(sc: SparkContext) = { - this(sc, new CacheManager, SQLContext.createListenerAndUI(sc), true, new InMemoryCatalog) + this(new SharedState(sc), true) } def this(sparkContext: JavaSparkContext) = this(sparkContext.sc) @@ -100,20 +97,20 @@ class SQLContext private[sql]( } } + def sparkContext: SparkContext = sharedState.sparkContext + + protected[sql] def cacheManager: CacheManager = sharedState.cacheManager + protected[sql] def listener: SQLListener = sharedState.listener + protected[sql] def externalCatalog: ExternalCatalog = sharedState.externalCatalog + /** - * Returns a SQLContext as new session, with separated SQL configurations, temporary tables, - * registered functions, but sharing the same SparkContext, CacheManager, SQLListener and SQLTab. + * Returns a [[SQLContext]] as new session, with separated SQL configurations, temporary + * tables, registered functions, but sharing the same [[SparkContext]], cached data and + * other things. * * @since 1.6.0 */ - def newSession(): SQLContext = { - new SQLContext( - sparkContext = sparkContext, - cacheManager = cacheManager, - listener = listener, - isRootContext = false, - externalCatalog = externalCatalog) - } + def newSession(): SQLContext = new SQLContext(sharedState, isRootContext = false) /** * Per-session state, e.g. configuration, functions, temporary tables etc. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index c30f879dedaa1..d404a7c0aef59 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -22,10 +22,8 @@ import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry} import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.parser.ParserInterface -import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources.{DataSourceAnalysis, PreInsertCastAndRename, ResolveDataSource} -import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange} import org.apache.spark.sql.util.ExecutionListenerManager /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala new file mode 100644 index 0000000000000..9a30c7de1f8f2 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -0,0 +1,47 @@ +/* + * 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.internal + +import org.apache.spark.SparkContext +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.catalyst.catalog.{ExternalCatalog, InMemoryCatalog} +import org.apache.spark.sql.execution.CacheManager +import org.apache.spark.sql.execution.ui.SQLListener + + +/** + * A class that holds all state shared across sessions in a given [[SQLContext]]. + */ +private[sql] class SharedState(val sparkContext: SparkContext) { + + /** + * Class for caching query results reused in future executions. + */ + val cacheManager: CacheManager = new CacheManager + + /** + * A listener for SQL-specific [[org.apache.spark.scheduler.SparkListenerEvent]]s. + */ + val listener: SQLListener = SQLContext.createListenerAndUI(sparkContext) + + /** + * A catalog that interacts with external systems. + */ + lazy val externalCatalog: ExternalCatalog = new InMemoryCatalog + +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 42cda0be161d3..71ef99a6a9952 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -45,12 +45,10 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Expression, LeafExpression} import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.{ExecutedCommand, SetCommand} -import org.apache.spark.sql.execution.ui.SQLListener import org.apache.spark.sql.hive.client._ import org.apache.spark.sql.hive.execution.{AnalyzeTable, DescribeHiveTableCommand, HiveNativeCommand} -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.{SharedState, SQLConf} import org.apache.spark.sql.internal.SQLConf._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -63,32 +61,14 @@ import org.apache.spark.util.Utils * @since 1.0.0 */ class HiveContext private[hive]( - sc: SparkContext, - cacheManager: CacheManager, - listener: SQLListener, - @transient private[hive] val executionHive: HiveClientImpl, - @transient private[hive] val metadataHive: HiveClient, - isRootContext: Boolean, - @transient private[sql] val hiveCatalog: HiveExternalCatalog) - extends SQLContext(sc, cacheManager, listener, isRootContext, hiveCatalog) with Logging { - self => + @transient protected[hive] val hiveSharedState: HiveSharedState, + override val isRootContext: Boolean) + extends SQLContext(hiveSharedState, isRootContext) with Logging { - private def this(sc: SparkContext, execHive: HiveClientImpl, metaHive: HiveClient) { - this( - sc, - new CacheManager, - SQLContext.createListenerAndUI(sc), - execHive, - metaHive, - true, - new HiveExternalCatalog(metaHive)) - } + self => def this(sc: SparkContext) = { - this( - sc, - HiveContext.newClientForExecution(sc.conf, sc.hadoopConfiguration), - HiveContext.newClientForMetadata(sc.conf, sc.hadoopConfiguration)) + this(new HiveSharedState(sc), true) } def this(sc: JavaSparkContext) = this(sc.sc) @@ -103,19 +83,16 @@ class HiveContext private[hive]( * and Hive client (both of execution and metadata) with existing HiveContext. */ override def newSession(): HiveContext = { - new HiveContext( - sc = sc, - cacheManager = cacheManager, - listener = listener, - executionHive = executionHive.newSession(), - metadataHive = metadataHive.newSession(), - isRootContext = false, - hiveCatalog = hiveCatalog) + new HiveContext(hiveSharedState, isRootContext = false) } @transient protected[sql] override lazy val sessionState = new HiveSessionState(self) + protected[hive] def hiveCatalog: HiveExternalCatalog = hiveSharedState.externalCatalog + protected[hive] def executionHive: HiveClientImpl = sessionState.executionHive + protected[hive] def metadataHive: HiveClient = sessionState.metadataHive + /** * When true, enables an experimental feature where metastore tables that use the parquet SerDe * are automatically converted to use the Spark SQL parquet table scan, instead of the Hive @@ -159,7 +136,7 @@ class HiveContext private[hive]( protected[hive] def hiveThriftServerAsync: Boolean = getConf(HIVE_THRIFT_SERVER_ASYNC) protected[hive] def hiveThriftServerSingleSession: Boolean = - sc.conf.get("spark.sql.hive.thriftServer.singleSession", "false").toBoolean + sparkContext.conf.getBoolean("spark.sql.hive.thriftServer.singleSession", defaultValue = false) @transient protected[sql] lazy val substitutor = new VariableSubstitution() @@ -527,7 +504,9 @@ private[hive] object HiveContext extends Logging { * The version of the Hive client that is used here must match the metastore that is configured * in the hive-site.xml file. */ - private def newClientForMetadata(conf: SparkConf, hadoopConf: Configuration): HiveClient = { + protected[hive] def newClientForMetadata( + conf: SparkConf, + hadoopConf: Configuration): HiveClient = { val hiveConf = new HiveConf(hadoopConf, classOf[HiveConf]) val configurations = hiveClientConfigurations(hiveConf) newClientForMetadata(conf, hiveConf, hadoopConf, configurations) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala index b992fda18cef7..bc28b55d06d9e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala @@ -18,10 +18,11 @@ package org.apache.spark.sql.hive import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry} +import org.apache.spark.sql.catalyst.analysis.Analyzer import org.apache.spark.sql.catalyst.parser.ParserInterface -import org.apache.spark.sql.execution.{python, SparkPlanner} +import org.apache.spark.sql.execution.SparkPlanner import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.hive.client.{HiveClient, HiveClientImpl} import org.apache.spark.sql.hive.execution.HiveSqlParser import org.apache.spark.sql.internal.{SessionState, SQLConf} @@ -31,6 +32,16 @@ import org.apache.spark.sql.internal.{SessionState, SQLConf} */ private[hive] class HiveSessionState(ctx: HiveContext) extends SessionState(ctx) { + /** + * A Hive client used for execution. + */ + val executionHive: HiveClientImpl = ctx.hiveSharedState.executionHive.newSession() + + /** + * A Hive client used for interacting with the metastore. + */ + val metadataHive: HiveClient = ctx.hiveSharedState.metadataHive.newSession() + override lazy val conf: SQLConf = new SQLConf { override def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE, false) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala new file mode 100644 index 0000000000000..11097c33df2d5 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.apache.spark.SparkContext +import org.apache.spark.sql.hive.client.{HiveClient, HiveClientImpl} +import org.apache.spark.sql.internal.SharedState + + +/** + * A class that holds all state shared across sessions in a given [[HiveContext]]. + */ +private[hive] class HiveSharedState(override val sparkContext: SparkContext) + extends SharedState(sparkContext) { + + // TODO: just share the IsolatedClientLoader instead of the client instances themselves + + /** + * A Hive client used for execution. + */ + val executionHive: HiveClientImpl = { + HiveContext.newClientForExecution(sparkContext.conf, sparkContext.hadoopConfiguration) + } + + /** + * A Hive client used to interact with the metastore. + */ + // This needs to be a lazy val at here because TestHiveSharedState is overriding it. + lazy val metadataHive: HiveClient = { + HiveContext.newClientForMetadata(sparkContext.conf, sparkContext.hadoopConfiguration) + } + + /** + * A catalog that interacts with the Hive metastore. + */ + override lazy val externalCatalog = new HiveExternalCatalog(metadataHive) + +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 7f6ca21782da4..d56d36fe32e77 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -72,63 +72,24 @@ object TestHive * test cases that rely on TestHive must be serialized. */ class TestHiveContext private[hive]( - sc: SparkContext, - cacheManager: CacheManager, - listener: SQLListener, - executionHive: HiveClientImpl, - metadataHive: HiveClient, - isRootContext: Boolean, - hiveCatalog: HiveExternalCatalog, + testHiveSharedState: TestHiveSharedState, val warehousePath: File, val scratchDirPath: File, - metastoreTemporaryConf: Map[String, String]) - extends HiveContext( - sc, - cacheManager, - listener, - executionHive, - metadataHive, - isRootContext, - hiveCatalog) { self => - - // Unfortunately, due to the complex interactions between the construction parameters - // and the limitations in scala constructors, we need many of these constructors to - // provide a shorthand to create a new TestHiveContext with only a SparkContext. - // This is not a great design pattern but it's necessary here. + metastoreTemporaryConf: Map[String, String], + isRootContext: Boolean) + extends HiveContext(testHiveSharedState, isRootContext) { self => private def this( sc: SparkContext, - executionHive: HiveClientImpl, - metadataHive: HiveClient, warehousePath: File, scratchDirPath: File, metastoreTemporaryConf: Map[String, String]) { this( - sc, - new CacheManager, - SQLContext.createListenerAndUI(sc), - executionHive, - metadataHive, - true, - new HiveExternalCatalog(metadataHive), + new TestHiveSharedState(sc, warehousePath, scratchDirPath, metastoreTemporaryConf), warehousePath, scratchDirPath, - metastoreTemporaryConf) - } - - private def this( - sc: SparkContext, - warehousePath: File, - scratchDirPath: File, - metastoreTemporaryConf: Map[String, String]) { - this( - sc, - HiveContext.newClientForExecution(sc.conf, sc.hadoopConfiguration), - TestHiveContext.newClientForMetadata( - sc.conf, sc.hadoopConfiguration, warehousePath, scratchDirPath, metastoreTemporaryConf), - warehousePath, - scratchDirPath, - metastoreTemporaryConf) + metastoreTemporaryConf, + true) } def this(sc: SparkContext) { @@ -141,16 +102,11 @@ class TestHiveContext private[hive]( override def newSession(): HiveContext = { new TestHiveContext( - sc = sc, - cacheManager = cacheManager, - listener = listener, - executionHive = executionHive.newSession(), - metadataHive = metadataHive.newSession(), - isRootContext = false, - hiveCatalog = hiveCatalog, - warehousePath = warehousePath, - scratchDirPath = scratchDirPath, - metastoreTemporaryConf = metastoreTemporaryConf) + testHiveSharedState, + warehousePath, + scratchDirPath, + metastoreTemporaryConf, + isRootContext = false) } // By clearing the port we force Spark to pick a new one. This allows us to rerun tests @@ -549,6 +505,22 @@ private[hive] class TestHiveFunctionRegistry extends SimpleFunctionRegistry { } } + +private[hive] class TestHiveSharedState( + sc: SparkContext, + warehousePath: File, + scratchDirPath: File, + metastoreTemporaryConf: Map[String, String]) + extends HiveSharedState(sc) { + + override lazy val metadataHive: HiveClient = { + TestHiveContext.newClientForMetadata( + sc.conf, sc.hadoopConfiguration, warehousePath, scratchDirPath, metastoreTemporaryConf) + } + +} + + private[hive] object TestHiveContext { /** @@ -563,7 +535,7 @@ private[hive] object TestHiveContext { /** * Create a [[HiveClient]] used to retrieve metadata from the Hive MetaStore. */ - private def newClientForMetadata( + def newClientForMetadata( conf: SparkConf, hadoopConf: Configuration, warehousePath: File, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala index 3334c16f0be87..84285b7f40832 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala @@ -18,12 +18,10 @@ package org.apache.spark.sql.hive import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.util.VersionInfo import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.catalog._ -import org.apache.spark.sql.hive.client.{HiveClient, IsolatedClientLoader} -import org.apache.spark.util.Utils +import org.apache.spark.sql.hive.client.HiveClient /** * Test suite for the [[HiveExternalCatalog]]. @@ -31,11 +29,9 @@ import org.apache.spark.util.Utils class HiveExternalCatalogSuite extends CatalogTestCases { private val client: HiveClient = { - IsolatedClientLoader.forVersion( - hiveMetastoreVersion = HiveContext.hiveExecutionVersion, - hadoopVersion = VersionInfo.getVersion, - sparkConf = new SparkConf(), - hadoopConf = new Configuration()).createClient() + // We create a metastore at a temp location to avoid any potential + // conflict of having multiple connections to a single derby instance. + HiveContext.newClientForExecution(new SparkConf, new Configuration) } protected override val utils: CatalogTestUtils = new CatalogTestUtils { From f31a62d1b24aea8ddfa40b60378ce065518786e4 Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Mon, 18 Apr 2016 13:31:48 -0700 Subject: [PATCH 09/69] [SPARK-14440][PYSPARK] Remove pipeline specific reader and writer ## What changes were proposed in this pull request? https://issues.apache.org/jira/browse/SPARK-14440 Remove * PipelineMLWriter * PipelineMLReader * PipelineModelMLWriter * PipelineModelMLReader and modify comments. ## How was this patch tested? test with unit test. Author: Xusen Yin Closes #12216 from yinxusen/SPARK-14440. --- python/pyspark/ml/pipeline.py | 53 +++++------------------------------ 1 file changed, 7 insertions(+), 46 deletions(-) diff --git a/python/pyspark/ml/pipeline.py b/python/pyspark/ml/pipeline.py index 6f599b51596fb..e2651aebdfd71 100644 --- a/python/pyspark/ml/pipeline.py +++ b/python/pyspark/ml/pipeline.py @@ -29,24 +29,6 @@ from pyspark.mllib.common import inherit_doc -@inherit_doc -class PipelineMLWriter(JavaMLWriter): - """ - Private Pipeline utility class that can save ML instances through their Scala implementation. - - We can currently use JavaMLWriter, rather than MLWriter, since Pipeline implements _to_java. - """ - - -@inherit_doc -class PipelineMLReader(JavaMLReader): - """ - Private utility class that can load Pipeline instances through their Scala implementation. - - We can currently use JavaMLReader, rather than MLReader, since Pipeline implements _from_java. - """ - - @inherit_doc class Pipeline(Estimator, MLReadable, MLWritable): """ @@ -154,8 +136,8 @@ def copy(self, extra=None): @since("2.0.0") def write(self): - """Returns an JavaMLWriter instance for this ML instance.""" - return PipelineMLWriter(self) + """Returns an MLWriter instance for this ML instance.""" + return JavaMLWriter(self) @since("2.0.0") def save(self, path): @@ -166,7 +148,7 @@ def save(self, path): @since("2.0.0") def read(cls): """Returns an MLReader instance for this class.""" - return PipelineMLReader(cls) + return JavaMLReader(cls) @classmethod def _from_java(cls, java_stage): @@ -201,27 +183,6 @@ def _to_java(self): return _java_obj -@inherit_doc -class PipelineModelMLWriter(JavaMLWriter): - """ - Private PipelineModel utility class that can save ML instances through their Scala - implementation. - - We can (currently) use JavaMLWriter, rather than MLWriter, since PipelineModel implements - _to_java. - """ - - -@inherit_doc -class PipelineModelMLReader(JavaMLReader): - """ - Private utility class that can load PipelineModel instances through their Scala implementation. - - We can currently use JavaMLReader, rather than MLReader, since PipelineModel implements - _from_java. - """ - - @inherit_doc class PipelineModel(Model, MLReadable, MLWritable): """ @@ -254,8 +215,8 @@ def copy(self, extra=None): @since("2.0.0") def write(self): - """Returns an JavaMLWriter instance for this ML instance.""" - return PipelineModelMLWriter(self) + """Returns an MLWriter instance for this ML instance.""" + return JavaMLWriter(self) @since("2.0.0") def save(self, path): @@ -265,8 +226,8 @@ def save(self, path): @classmethod @since("2.0.0") def read(cls): - """Returns an JavaMLReader instance for this class.""" - return PipelineModelMLReader(cls) + """Returns an MLReader instance for this class.""" + return JavaMLReader(cls) @classmethod def _from_java(cls, java_stage): From 8c62edb70fdeedf0ca5a7fc154698aea96184cc6 Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Mon, 18 Apr 2016 13:34:36 -0700 Subject: [PATCH 10/69] [SPARK-14299][EXAMPLES] Remove duplications for scala.examples.ml ## What changes were proposed in this pull request? https://issues.apache.org/jira/browse/SPARK-14299 Delete duplications in scala/examples/ml. TrainValidationSplitExample.scala --> ModelSelectionViaTrainValidationSplitExample CrossValidatorExample.scala --> ModelSelectionViaCrossValidationExample ## How was this patch tested? Existing tests passed. (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Author: Xusen Yin Closes #12366 from yinxusen/SPARK-14299-2. --- .../examples/ml/CrossValidatorExample.scala | 114 ------------------ ...elSelectionViaCrossValidationExample.scala | 9 ++ ...ectionViaTrainValidationSplitExample.scala | 8 ++ .../ml/TrainValidationSplitExample.scala | 78 ------------ 4 files changed, 17 insertions(+), 192 deletions(-) delete mode 100644 examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala delete mode 100644 examples/src/main/scala/org/apache/spark/examples/ml/TrainValidationSplitExample.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala deleted file mode 100644 index bca301d412f4c..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala +++ /dev/null @@ -1,114 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -// scalastyle:off println -package org.apache.spark.examples.ml - -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.ml.Pipeline -import org.apache.spark.ml.classification.LogisticRegression -import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator -import org.apache.spark.ml.feature.{HashingTF, Tokenizer} -import org.apache.spark.ml.tuning.{CrossValidator, ParamGridBuilder} -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.sql.{Row, SQLContext} - -/** - * A simple example demonstrating model selection using CrossValidator. - * This example also demonstrates how Pipelines are Estimators. - * - * This example uses the [[LabeledDocument]] and [[Document]] case classes from - * [[SimpleTextClassificationPipeline]]. - * - * Run with - * {{{ - * bin/run-example ml.CrossValidatorExample - * }}} - */ -object CrossValidatorExample { - - def main(args: Array[String]) { - val conf = new SparkConf().setAppName("CrossValidatorExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - import sqlContext.implicits._ - - // Prepare training documents, which are labeled. - val training = sc.parallelize(Seq( - LabeledDocument(0L, "a b c d e spark", 1.0), - LabeledDocument(1L, "b d", 0.0), - LabeledDocument(2L, "spark f g h", 1.0), - LabeledDocument(3L, "hadoop mapreduce", 0.0), - LabeledDocument(4L, "b spark who", 1.0), - LabeledDocument(5L, "g d a y", 0.0), - LabeledDocument(6L, "spark fly", 1.0), - LabeledDocument(7L, "was mapreduce", 0.0), - LabeledDocument(8L, "e spark program", 1.0), - LabeledDocument(9L, "a e c l", 0.0), - LabeledDocument(10L, "spark compile", 1.0), - LabeledDocument(11L, "hadoop software", 0.0))) - - // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. - val tokenizer = new Tokenizer() - .setInputCol("text") - .setOutputCol("words") - val hashingTF = new HashingTF() - .setInputCol(tokenizer.getOutputCol) - .setOutputCol("features") - val lr = new LogisticRegression() - .setMaxIter(10) - val pipeline = new Pipeline() - .setStages(Array(tokenizer, hashingTF, lr)) - - // We now treat the Pipeline as an Estimator, wrapping it in a CrossValidator instance. - // This will allow us to jointly choose parameters for all Pipeline stages. - // A CrossValidator requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. - val crossval = new CrossValidator() - .setEstimator(pipeline) - .setEvaluator(new BinaryClassificationEvaluator) - // We use a ParamGridBuilder to construct a grid of parameters to search over. - // With 3 values for hashingTF.numFeatures and 2 values for lr.regParam, - // this grid will have 3 x 2 = 6 parameter settings for CrossValidator to choose from. - val paramGrid = new ParamGridBuilder() - .addGrid(hashingTF.numFeatures, Array(10, 100, 1000)) - .addGrid(lr.regParam, Array(0.1, 0.01)) - .build() - crossval.setEstimatorParamMaps(paramGrid) - crossval.setNumFolds(2) // Use 3+ in practice - - // Run cross-validation, and choose the best set of parameters. - val cvModel = crossval.fit(training.toDF()) - - // Prepare test documents, which are unlabeled. - val test = sc.parallelize(Seq( - Document(4L, "spark i j k"), - Document(5L, "l m n"), - Document(6L, "mapreduce spark"), - Document(7L, "apache hadoop"))) - - // Make predictions on test documents. cvModel uses the best model found (lrModel). - cvModel.transform(test.toDF()) - .select("id", "text", "probability", "prediction") - .collect() - .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) => - println(s"($id, $text) --> prob=$prob, prediction=$prediction") - } - - sc.stop() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala index 0331d6e7b35df..d1441b5497a86 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaCrossValidationExample.scala @@ -30,6 +30,15 @@ import org.apache.spark.sql.Row // $example off$ import org.apache.spark.sql.SQLContext +/** + * A simple example demonstrating model selection using CrossValidator. + * This example also demonstrates how Pipelines are Estimators. + * + * Run with + * {{{ + * bin/run-example ml.ModelSelectionViaCrossValidationExample + * }}} + */ object ModelSelectionViaCrossValidationExample { def main(args: Array[String]): Unit = { diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala index 5a95344f223df..fcad17a817580 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/ModelSelectionViaTrainValidationSplitExample.scala @@ -25,6 +25,14 @@ import org.apache.spark.ml.tuning.{ParamGridBuilder, TrainValidationSplit} // $example off$ import org.apache.spark.sql.SQLContext +/** + * A simple example demonstrating model selection using TrainValidationSplit. + * + * Run with + * {{{ + * bin/run-example ml.ModelSelectionViaTrainValidationSplitExample + * }}} + */ object ModelSelectionViaTrainValidationSplitExample { def main(args: Array[String]): Unit = { diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/TrainValidationSplitExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/TrainValidationSplitExample.scala deleted file mode 100644 index fbba17eba6a2f..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/ml/TrainValidationSplitExample.scala +++ /dev/null @@ -1,78 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.examples.ml - -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.ml.evaluation.RegressionEvaluator -import org.apache.spark.ml.regression.LinearRegression -import org.apache.spark.ml.tuning.{ParamGridBuilder, TrainValidationSplit} -import org.apache.spark.sql.SQLContext - -/** - * A simple example demonstrating model selection using TrainValidationSplit. - * - * The example is based on [[SimpleParamsExample]] using linear regression. - * Run with - * {{{ - * bin/run-example ml.TrainValidationSplitExample - * }}} - */ -object TrainValidationSplitExample { - - def main(args: Array[String]): Unit = { - val conf = new SparkConf().setAppName("TrainValidationSplitExample") - val sc = new SparkContext(conf) - val sqlContext = new SQLContext(sc) - - // Prepare training and test data. - val data = sqlContext.read.format("libsvm").load("data/mllib/sample_libsvm_data.txt") - val Array(training, test) = data.randomSplit(Array(0.9, 0.1), seed = 12345) - - val lr = new LinearRegression() - - // We use a ParamGridBuilder to construct a grid of parameters to search over. - // TrainValidationSplit will try all combinations of values and determine best model using - // the evaluator. - val paramGrid = new ParamGridBuilder() - .addGrid(lr.regParam, Array(0.1, 0.01)) - .addGrid(lr.fitIntercept, Array(true, false)) - .addGrid(lr.elasticNetParam, Array(0.0, 0.5, 1.0)) - .build() - - // In this case the estimator is simply the linear regression. - // A TrainValidationSplit requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. - val trainValidationSplit = new TrainValidationSplit() - .setEstimator(lr) - .setEvaluator(new RegressionEvaluator) - .setEstimatorParamMaps(paramGrid) - - // 80% of the data will be used for training and the remaining 20% for validation. - trainValidationSplit.setTrainRatio(0.8) - - // Run train validation split, and choose the best set of parameters. - val model = trainValidationSplit.fit(training) - - // Make predictions on test data. model is the model with combination of parameters - // that performed best. - model.transform(test) - .select("features", "label", "prediction") - .show() - - sc.stop() - } -} From 6fc1e72d9b70615bd91b598084406eb1893d6706 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Mon, 18 Apr 2016 13:45:03 -0700 Subject: [PATCH 11/69] [MINOR] Revert removing explicit typing (changed in some examples and StatFunctions) ## What changes were proposed in this pull request? This PR reverts some changes in https://github.com/apache/spark/pull/12413. (please see the discussion in that PR). from ```scala words.foreachRDD { (rdd, time) => ... ``` to ```scala words.foreachRDD { (rdd: RDD[String], time: Time) => ... ``` Also, this was discussed in dev-mailing list, [here](http://apache-spark-developers-list.1001551.n3.nabble.com/Question-about-Scala-style-explicit-typing-within-transformation-functions-and-anonymous-val-td17173.html) ## How was this patch tested? This was tested with `sbt scalastyle`. Author: hyukjinkwon Closes #12452 from HyukjinKwon/revert-explicit-typing. --- .../spark/examples/streaming/RecoverableNetworkWordCount.scala | 2 +- .../apache/spark/examples/streaming/SqlNetworkWordCount.scala | 2 +- .../org/apache/spark/sql/execution/stat/StatFunctions.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala index aa762b27dc3bb..1bcd85e1d533f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala @@ -116,7 +116,7 @@ object RecoverableNetworkWordCount { val lines = ssc.socketTextStream(ip, port) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map((_, 1)).reduceByKey(_ + _) - wordCounts.foreachRDD { (rdd, time) => + wordCounts.foreachRDD { (rdd: RDD[(String, Int)], time: Time) => // Get or register the blacklist Broadcast val blacklist = WordBlacklist.getInstance(rdd.sparkContext) // Get or register the droppedWordsCounter Accumulator diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala index ad6a89e320f56..918e124065e4c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala @@ -59,7 +59,7 @@ object SqlNetworkWordCount { val words = lines.flatMap(_.split(" ")) // Convert RDDs of the words DStream to DataFrame and run SQL query - words.foreachRDD { (rdd, time) => + words.foreachRDD { (rdd: RDD[String], time: Time) => // Get the singleton instance of SQLContext val sqlContext = SQLContextSingleton.getInstance(rdd.sparkContext) import sqlContext.implicits._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala index 9afbd0e994ff4..d603f63a08501 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala @@ -431,7 +431,7 @@ private[sql] object StatFunctions extends Logging { s"exceed 1e4. Currently $columnSize") val table = counts.groupBy(_.get(0)).map { case (col1Item, rows) => val countsRow = new GenericMutableRow(columnSize + 1) - rows.foreach { row => + rows.foreach { (row: Row) => // row.get(0) is column 1 // row.get(1) is column 2 // row.get(2) is the frequency From 8bd8121329cb1bb137e935dec124aa23f0fcf8c5 Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Mon, 18 Apr 2016 14:03:40 -0700 Subject: [PATCH 12/69] [SPARK-14710][SQL] Rename gen/genCode to genCode/doGenCode to better reflect the semantics ## What changes were proposed in this pull request? Per rxin's suggestions, this patch renames `s/gen/genCode` and `s/genCode/doGenCode` to better reflect the semantics of these 2 function calls. ## How was this patch tested? N/A (refactoring only) Author: Sameer Agarwal Closes #12475 from sameeragarwal/gencode. --- .../sql/catalyst/analysis/unresolved.scala | 2 +- .../catalyst/expressions/BoundAttribute.scala | 2 +- .../spark/sql/catalyst/expressions/Cast.scala | 4 +- .../sql/catalyst/expressions/Expression.scala | 28 +++---- .../catalyst/expressions/InputFileName.scala | 2 +- .../MonotonicallyIncreasingID.scala | 2 +- .../expressions/ReferenceToExpressions.scala | 6 +- .../sql/catalyst/expressions/ScalaUDF.scala | 4 +- .../sql/catalyst/expressions/SortOrder.scala | 4 +- .../expressions/SparkPartitionID.scala | 2 +- .../sql/catalyst/expressions/TimeWindow.scala | 4 +- .../sql/catalyst/expressions/arithmetic.scala | 38 ++++----- .../expressions/bitwiseExpressions.scala | 2 +- .../expressions/codegen/CodeGenerator.scala | 4 +- .../expressions/codegen/CodegenFallback.scala | 2 +- .../codegen/GenerateOrdering.scala | 2 +- .../codegen/GeneratePredicate.scala | 2 +- .../codegen/GenerateSafeProjection.scala | 2 +- .../expressions/collectionOperations.scala | 4 +- .../expressions/complexTypeCreator.scala | 22 ++--- .../expressions/complexTypeExtractors.scala | 8 +- .../expressions/conditionalExpressions.scala | 27 ++++--- .../expressions/datetimeExpressions.scala | 64 +++++++-------- .../expressions/decimalExpressions.scala | 10 +-- .../sql/catalyst/expressions/literals.scala | 8 +- .../expressions/mathExpressions.scala | 38 ++++----- .../spark/sql/catalyst/expressions/misc.scala | 18 ++--- .../expressions/namedExpressions.scala | 4 +- .../expressions/nullExpressions.scala | 28 +++---- .../sql/catalyst/expressions/objects.scala | 57 ++++++------- .../sql/catalyst/expressions/predicates.scala | 34 ++++---- .../expressions/randomExpressions.scala | 4 +- .../expressions/regexpExpressions.scala | 14 ++-- .../expressions/stringExpressions.scala | 80 +++++++++---------- .../expressions/NonFoldableLiteral.scala | 4 +- .../spark/sql/execution/ExistingRDD.scala | 2 +- .../apache/spark/sql/execution/Expand.scala | 4 +- .../sql/execution/WholeStageCodegen.scala | 2 +- .../aggregate/TungstenAggregate.scala | 27 ++++--- .../spark/sql/execution/basicOperators.scala | 4 +- .../execution/joins/BroadcastHashJoin.scala | 10 ++- .../sql/execution/joins/SortMergeJoin.scala | 6 +- .../apache/spark/sql/execution/objects.scala | 8 +- .../apache/spark/sql/execution/subquery.scala | 4 +- 44 files changed, 305 insertions(+), 298 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 4ec43aba02d66..90b7b60b1c216 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -153,7 +153,7 @@ case class UnresolvedGenerator(name: String, children: Seq[Expression]) extends override def eval(input: InternalRow = null): TraversableOnce[InternalRow] = throw new UnsupportedOperationException(s"Cannot evaluate expression: $this") - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = throw new UnsupportedOperationException(s"Cannot evaluate expression: $this") override def terminate(): TraversableOnce[InternalRow] = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index c1fd23f28d6b3..cf23884c44124 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -58,7 +58,7 @@ case class BoundReference(ordinal: Int, dataType: DataType, nullable: Boolean) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val javaType = ctx.javaType(dataType) val value = ctx.getValue(ctx.INPUT_ROW, dataType, ordinal.toString) if (ctx.currentVars != null && ctx.currentVars(ordinal) != null) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 0f8876a9e6881..ffb100ee54d23 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -446,8 +446,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w protected override def nullSafeEval(input: Any): Any = cast(input) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval = child.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval = child.genCode(ctx) val nullSafeCast = nullSafeCastFunction(child.dataType, dataType, ctx) eval.code + castCode(ctx, eval.value, eval.isNull, ev.value, ev.isNull, dataType, nullSafeCast) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 718bb4b118cea..11e3fd78d0799 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -86,23 +86,23 @@ abstract class Expression extends TreeNode[Expression] { def eval(input: InternalRow = null): Any /** - * Returns an [[ExprCode]], which contains Java source code that - * can be used to generate the result of evaluating the expression on an input row. + * Returns an [[ExprCode]], that contains the Java source code to generate the result of + * evaluating the expression on an input row. * * @param ctx a [[CodegenContext]] * @return [[ExprCode]] */ - def gen(ctx: CodegenContext): ExprCode = { + def genCode(ctx: CodegenContext): ExprCode = { ctx.subExprEliminationExprs.get(this).map { subExprState => - // This expression is repeated meaning the code to evaluated has already been added - // as a function and called in advance. Just use it. + // This expression is repeated which means that the code to evaluate it has already been added + // as a function before. In that case, we just re-use it. val code = s"/* ${toCommentSafeString(this.toString)} */" ExprCode(code, subExprState.isNull, subExprState.value) }.getOrElse { val isNull = ctx.freshName("isNull") val value = ctx.freshName("value") val ve = ExprCode("", isNull, value) - ve.code = genCode(ctx, ve) + ve.code = doGenCode(ctx, ve) if (ve.code != "") { // Add `this` in the comment. ve.copy(s"/* ${toCommentSafeString(this.toString)} */\n" + ve.code.trim) @@ -121,7 +121,7 @@ abstract class Expression extends TreeNode[Expression] { * @param ev an [[ExprCode]] with unique terms. * @return Java source code */ - protected def genCode(ctx: CodegenContext, ev: ExprCode): String + protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String /** * Returns `true` if this expression and all its children have been resolved to a specific schema @@ -216,7 +216,7 @@ trait Unevaluable extends Expression { final override def eval(input: InternalRow = null): Any = throw new UnsupportedOperationException(s"Cannot evaluate expression: $this") - final override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = + final override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = throw new UnsupportedOperationException(s"Cannot evaluate expression: $this") } @@ -333,7 +333,7 @@ abstract class UnaryExpression extends Expression { ctx: CodegenContext, ev: ExprCode, f: String => String): String = { - val childGen = child.gen(ctx) + val childGen = child.genCode(ctx) val resultCode = f(childGen.value) if (nullable) { @@ -424,8 +424,8 @@ abstract class BinaryExpression extends Expression { ctx: CodegenContext, ev: ExprCode, f: (String, String) => String): String = { - val leftGen = left.gen(ctx) - val rightGen = right.gen(ctx) + val leftGen = left.genCode(ctx) + val rightGen = right.genCode(ctx) val resultCode = f(leftGen.value, rightGen.value) if (nullable) { @@ -566,9 +566,9 @@ abstract class TernaryExpression extends Expression { ctx: CodegenContext, ev: ExprCode, f: (String, String, String) => String): String = { - val leftGen = children(0).gen(ctx) - val midGen = children(1).gen(ctx) - val rightGen = children(2).gen(ctx) + val leftGen = children(0).genCode(ctx) + val midGen = children(1).genCode(ctx) + val rightGen = children(2).genCode(ctx) val resultCode = f(leftGen.value, midGen.value, rightGen.value) if (nullable) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InputFileName.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InputFileName.scala index 2ed6fc0d3824f..144efb751bf7d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InputFileName.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InputFileName.scala @@ -43,7 +43,7 @@ case class InputFileName() extends LeafExpression with Nondeterministic { InputFileNameHolder.getInputFileName() } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { ev.isNull = "false" s"final ${ctx.javaType(dataType)} ${ev.value} = " + "org.apache.spark.rdd.InputFileNameHolder.getInputFileName();" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala index 5d28f8fbde8be..9d3e80cad69dd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala @@ -65,7 +65,7 @@ private[sql] case class MonotonicallyIncreasingID() extends LeafExpression with partitionMask + currentCount } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val countTerm = ctx.freshName("count") val partitionMaskTerm = ctx.freshName("partitionMask") ctx.addMutableState(ctx.JAVA_LONG, countTerm, s"$countTerm = 0L;") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala index 22645c952e722..98710f8d781d8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ReferenceToExpressions.scala @@ -59,15 +59,15 @@ case class ReferenceToExpressions(result: Expression, children: Seq[Expression]) result.eval(projection(input)) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val childrenGen = children.map(_.gen(ctx)) + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val childrenGen = children.map(_.genCode(ctx)) val childrenVars = childrenGen.zip(children).map { case (childGen, child) => LambdaVariable(childGen.value, childGen.isNull, child.dataType) } val resultGen = result.transform { case b: BoundReference => childrenVars(b.ordinal) - }.gen(ctx) + }.genCode(ctx) ev.value = resultGen.value ev.isNull = resultGen.isNull diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index 500ff447a9754..1b19cdbadd33c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -989,7 +989,7 @@ case class ScalaUDF( converterTerm } - override def genCode( + override def doGenCode( ctx: CodegenContext, ev: ExprCode): String = { @@ -1024,7 +1024,7 @@ case class ScalaUDF( s"[$funcExpressionIdx]).userDefinedFunc());") // codegen for children expressions - val evals = children.map(_.gen(ctx)) + val evals = children.map(_.genCode(ctx)) // Generate the codes for expressions and calling user-defined function // We need to get the boxedType of dataType's javaType here. Because for the dataType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala index b739361937b6b..beced2c646ea9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala @@ -70,8 +70,8 @@ case class SortPrefix(child: SortOrder) extends UnaryExpression { override def eval(input: InternalRow): Any = throw new UnsupportedOperationException - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val childCode = child.child.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val childCode = child.child.genCode(ctx) val input = childCode.value val BinaryPrefixCmp = classOf[BinaryPrefixComparator].getName val DoublePrefixCmp = classOf[DoublePrefixComparator].getName diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala index 377f08eb105fa..8ca168a85b27a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala @@ -44,7 +44,7 @@ private[sql] case class SparkPartitionID() extends LeafExpression with Nondeterm override protected def evalInternal(input: InternalRow): Int = partitionId - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val idTerm = ctx.freshName("partitionId") ctx.addMutableState(ctx.JAVA_INT, idTerm, s"$idTerm = org.apache.spark.TaskContext.getPartitionId();") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index daf3de95dd9ea..46cbd124963fe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -158,8 +158,8 @@ object TimeWindow { case class PreciseTimestamp(child: Expression) extends UnaryExpression with ExpectsInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) override def dataType: DataType = LongType - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval = child.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval = child.genCode(ctx) eval.code + s"""boolean ${ev.isNull} = ${eval.isNull}; |${ctx.javaType(dataType)} ${ev.value} = ${eval.value}; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index f3d42fc0b2164..25806c547b091 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -36,7 +36,7 @@ case class UnaryMinus(child: Expression) extends UnaryExpression private lazy val numeric = TypeUtils.getNumeric(dataType) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = dataType match { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = dataType match { case dt: DecimalType => defineCodeGen(ctx, ev, c => s"$c.unary_$$minus()") case dt: NumericType => nullSafeCodeGen(ctx, ev, eval => { val originValue = ctx.freshName("origin") @@ -70,7 +70,7 @@ case class UnaryPositive(child: Expression) override def dataType: DataType = child.dataType - override def genCode(ctx: CodegenContext, ev: ExprCode): String = + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = defineCodeGen(ctx, ev, c => c) protected override def nullSafeEval(input: Any): Any = input @@ -93,7 +93,7 @@ case class Abs(child: Expression) private lazy val numeric = TypeUtils.getNumeric(dataType) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = dataType match { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = dataType match { case dt: DecimalType => defineCodeGen(ctx, ev, c => s"$c.abs()") case dt: NumericType => @@ -113,7 +113,7 @@ abstract class BinaryArithmetic extends BinaryOperator { def decimalMethod: String = sys.error("BinaryArithmetics must override either decimalMethod or genCode") - override def genCode(ctx: CodegenContext, ev: ExprCode): String = dataType match { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = dataType match { case dt: DecimalType => defineCodeGen(ctx, ev, (eval1, eval2) => s"$eval1.$decimalMethod($eval2)") // byte and short are casted into int when add, minus, times or divide @@ -147,7 +147,7 @@ case class Add(left: Expression, right: Expression) extends BinaryArithmetic wit } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = dataType match { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = dataType match { case dt: DecimalType => defineCodeGen(ctx, ev, (eval1, eval2) => s"$eval1.$$plus($eval2)") case ByteType | ShortType => @@ -179,7 +179,7 @@ case class Subtract(left: Expression, right: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = dataType match { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = dataType match { case dt: DecimalType => defineCodeGen(ctx, ev, (eval1, eval2) => s"$eval1.$$minus($eval2)") case ByteType | ShortType => @@ -241,9 +241,9 @@ case class Divide(left: Expression, right: Expression) /** * Special case handling due to division by 0 => null. */ - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval1 = left.gen(ctx) - val eval2 = right.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval1 = left.genCode(ctx) + val eval2 = right.genCode(ctx) val isZero = if (dataType.isInstanceOf[DecimalType]) { s"${eval2.value}.isZero()" } else { @@ -320,9 +320,9 @@ case class Remainder(left: Expression, right: Expression) /** * Special case handling for x % 0 ==> null. */ - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval1 = left.gen(ctx) - val eval2 = right.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval1 = left.genCode(ctx) + val eval2 = right.genCode(ctx) val isZero = if (dataType.isInstanceOf[DecimalType]) { s"${eval2.value}.isZero()" } else { @@ -393,9 +393,9 @@ case class MaxOf(left: Expression, right: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval1 = left.gen(ctx) - val eval2 = right.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval1 = left.genCode(ctx) + val eval2 = right.genCode(ctx) val compCode = ctx.genComp(dataType, eval1.value, eval2.value) eval1.code + eval2.code + s""" @@ -449,9 +449,9 @@ case class MinOf(left: Expression, right: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval1 = left.gen(ctx) - val eval2 = right.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval1 = left.genCode(ctx) + val eval2 = right.genCode(ctx) val compCode = ctx.genComp(dataType, eval1.value, eval2.value) eval1.code + eval2.code + s""" @@ -503,7 +503,7 @@ case class Pmod(left: Expression, right: Expression) extends BinaryArithmetic wi case _: DecimalType => pmod(left.asInstanceOf[Decimal], right.asInstanceOf[Decimal]) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (eval1, eval2) => { dataType match { case dt: DecimalType => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala index a7e1cd66f24aa..8fd8a9bd4eef1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala @@ -130,7 +130,7 @@ case class BitwiseNot(child: Expression) extends UnaryExpression with ExpectsInp ((evalE: Long) => ~evalE).asInstanceOf[(Any) => Any] } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"(${ctx.javaType(dataType)}) ~($c)") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index f43626ca814a0..38ac13b208aab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -526,7 +526,7 @@ class CodegenContext { val value = s"${fnName}Value" // Generate the code for this expression tree and wrap it in a function. - val code = expr.gen(this) + val code = expr.genCode(this) val fn = s""" |private void $fnName(InternalRow $INPUT_ROW) { @@ -572,7 +572,7 @@ class CodegenContext { def generateExpressions(expressions: Seq[Expression], doSubexpressionElimination: Boolean = false): Seq[ExprCode] = { if (doSubexpressionElimination) subexpressionElimination(expressions) - expressions.map(e => e.gen(this)) + expressions.map(e => e.genCode(this)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenFallback.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenFallback.scala index 1365ee4b55634..1e446c498d948 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenFallback.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenFallback.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.util.toCommentSafeString */ trait CodegenFallback extends Expression { - protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { foreach { case n: Nondeterministic => n.setInitialValues() case _ => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala index 908c32de4d896..5635c91830f4a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala @@ -70,7 +70,7 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[InternalR */ def genComparisons(ctx: CodegenContext, ordering: Seq[SortOrder]): String = { val comparisons = ordering.map { order => - val eval = order.child.gen(ctx) + val eval = order.child.genCode(ctx) val asc = order.direction == Ascending val isNullA = ctx.freshName("isNullA") val primitiveA = ctx.freshName("primitiveA") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala index 58065d956f072..dd8e2a289a661 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala @@ -39,7 +39,7 @@ object GeneratePredicate extends CodeGenerator[Expression, (InternalRow) => Bool protected def create(predicate: Expression): ((InternalRow) => Boolean) = { val ctx = newCodeGenContext() - val eval = predicate.gen(ctx) + val eval = predicate.genCode(ctx) val code = s""" public SpecificPredicate generate(Object[] references) { return new SpecificPredicate(references); diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala index cf73e36d227c1..7be57aca333de 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala @@ -141,7 +141,7 @@ object GenerateSafeProjection extends CodeGenerator[Seq[Expression], Projection] val expressionCodes = expressions.zipWithIndex.map { case (NoOp, _) => "" case (e, i) => - val evaluationCode = e.gen(ctx) + val evaluationCode = e.genCode(ctx) val converter = convertToSafe(ctx, evaluationCode.value, e.dataType) evaluationCode.code + s""" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index ab790cf372d9e..8cb691c9b151e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -37,7 +37,7 @@ case class Size(child: Expression) extends UnaryExpression with ExpectsInputType case _: MapType => value.asInstanceOf[MapData].numElements() } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, c => s"${ev.value} = ($c).numElements();") } } @@ -180,7 +180,7 @@ case class ArrayContains(left: Expression, right: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (arr, value) => { val i = ctx.freshName("i") val getValue = ctx.getValue(arr, right.dataType, i) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index 74de4a776de89..a7a59d8784da9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -48,7 +48,7 @@ case class CreateArray(children: Seq[Expression]) extends Expression { new GenericArrayData(children.map(_.eval(input)).toArray) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val arrayClass = classOf[GenericArrayData].getName val values = ctx.freshName("values") s""" @@ -56,7 +56,7 @@ case class CreateArray(children: Seq[Expression]) extends Expression { final Object[] $values = new Object[${children.size}]; """ + children.zipWithIndex.map { case (e, i) => - val eval = e.gen(ctx) + val eval = e.genCode(ctx) eval.code + s""" if (${eval.isNull}) { $values[$i] = null; @@ -115,7 +115,7 @@ case class CreateMap(children: Seq[Expression]) extends Expression { new ArrayBasedMapData(new GenericArrayData(keyArray), new GenericArrayData(valueArray)) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val arrayClass = classOf[GenericArrayData].getName val mapClass = classOf[ArrayBasedMapData].getName val keyArray = ctx.freshName("keyArray") @@ -128,7 +128,7 @@ case class CreateMap(children: Seq[Expression]) extends Expression { final Object[] $valueArray = new Object[${values.size}]; """ + keys.zipWithIndex.map { case (key, i) => - val eval = key.gen(ctx) + val eval = key.genCode(ctx) s""" ${eval.code} if (${eval.isNull}) { @@ -139,7 +139,7 @@ case class CreateMap(children: Seq[Expression]) extends Expression { """ }.mkString("\n") + values.zipWithIndex.map { case (value, i) => - val eval = value.gen(ctx) + val eval = value.genCode(ctx) s""" ${eval.code} if (${eval.isNull}) { @@ -181,7 +181,7 @@ case class CreateStruct(children: Seq[Expression]) extends Expression { InternalRow(children.map(_.eval(input)): _*) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val rowClass = classOf[GenericInternalRow].getName val values = ctx.freshName("values") s""" @@ -189,7 +189,7 @@ case class CreateStruct(children: Seq[Expression]) extends Expression { final Object[] $values = new Object[${children.size}]; """ + children.zipWithIndex.map { case (e, i) => - val eval = e.gen(ctx) + val eval = e.genCode(ctx) eval.code + s""" if (${eval.isNull}) { $values[$i] = null; @@ -262,7 +262,7 @@ case class CreateNamedStruct(children: Seq[Expression]) extends Expression { InternalRow(valExprs.map(_.eval(input)): _*) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val rowClass = classOf[GenericInternalRow].getName val values = ctx.freshName("values") s""" @@ -270,7 +270,7 @@ case class CreateNamedStruct(children: Seq[Expression]) extends Expression { final Object[] $values = new Object[${valExprs.size}]; """ + valExprs.zipWithIndex.map { case (e, i) => - val eval = e.gen(ctx) + val eval = e.genCode(ctx) eval.code + s""" if (${eval.isNull}) { $values[$i] = null; @@ -314,7 +314,7 @@ case class CreateStructUnsafe(children: Seq[Expression]) extends Expression { InternalRow(children.map(_.eval(input)): _*) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val eval = GenerateUnsafeProjection.createCode(ctx, children) ev.isNull = eval.isNull ev.value = eval.value @@ -354,7 +354,7 @@ case class CreateNamedStructUnsafe(children: Seq[Expression]) extends Expression InternalRow(valExprs.map(_.eval(input)): _*) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val eval = GenerateUnsafeProjection.createCode(ctx, valExprs) ev.isNull = eval.isNull ev.value = eval.value diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala index c06dcc98674fd..b5ff9f55d51f6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala @@ -122,7 +122,7 @@ case class GetStructField(child: Expression, ordinal: Int, name: Option[String] protected override def nullSafeEval(input: Any): Any = input.asInstanceOf[InternalRow].get(ordinal, childSchema(ordinal).dataType) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, eval => { if (nullable) { s""" @@ -179,7 +179,7 @@ case class GetArrayStructFields( new GenericArrayData(result) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val arrayClass = classOf[GenericArrayData].getName nullSafeCodeGen(ctx, ev, eval => { val n = ctx.freshName("n") @@ -239,7 +239,7 @@ case class GetArrayItem(child: Expression, ordinal: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (eval1, eval2) => { val index = ctx.freshName("index") s""" @@ -302,7 +302,7 @@ case class GetMapValue(child: Expression, key: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val index = ctx.freshName("index") val length = ctx.freshName("length") val keys = ctx.freshName("keys") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index ae6a94842f7d0..a4c800a26cc52 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -55,10 +55,10 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val condEval = predicate.gen(ctx) - val trueEval = trueValue.gen(ctx) - val falseEval = falseValue.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val condEval = predicate.genCode(ctx) + val trueEval = trueValue.genCode(ctx) + val falseEval = falseValue.genCode(ctx) s""" ${condEval.code} @@ -147,11 +147,11 @@ case class CaseWhen(branches: Seq[(Expression, Expression)], elseValue: Option[E branches.length < CaseWhen.MAX_NUM_CASES_FOR_CODEGEN } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { if (!shouldCodegen) { // Fallback to interpreted mode if there are too many branches, as it may reach the // 64K limit (limit on bytecode size for a single function). - return super[CodegenFallback].genCode(ctx, ev) + return super[CodegenFallback].doGenCode(ctx, ev) } // Generate code that looks like: // @@ -172,8 +172,8 @@ case class CaseWhen(branches: Seq[(Expression, Expression)], elseValue: Option[E // } // } val cases = branches.map { case (condExpr, valueExpr) => - val cond = condExpr.gen(ctx) - val res = valueExpr.gen(ctx) + val cond = condExpr.genCode(ctx) + val res = valueExpr.genCode(ctx) s""" ${cond.code} if (!${cond.isNull} && ${cond.value}) { @@ -187,7 +187,7 @@ case class CaseWhen(branches: Seq[(Expression, Expression)], elseValue: Option[E var generatedCode = cases.mkString("", "\nelse {\n", "\nelse {\n") elseValue.foreach { elseExpr => - val res = elseExpr.gen(ctx) + val res = elseExpr.genCode(ctx) generatedCode += s""" ${res.code} @@ -230,6 +230,7 @@ object CaseWhen { /** * A factory method to facilitate the creation of this expression when used in parsers. + * * @param branches Expressions at even position are the branch conditions, and expressions at odd * position are branch values. */ @@ -297,8 +298,8 @@ case class Least(children: Seq[Expression]) extends Expression { }) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val evalChildren = children.map(_.gen(ctx)) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val evalChildren = children.map(_.genCode(ctx)) val first = evalChildren(0) val rest = evalChildren.drop(1) def updateEval(eval: ExprCode): String = { @@ -358,8 +359,8 @@ case class Greatest(children: Seq[Expression]) extends Expression { }) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val evalChildren = children.map(_.gen(ctx)) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val evalChildren = children.map(_.genCode(ctx)) val first = evalChildren(0) val rest = evalChildren.drop(1) def updateEval(eval: ExprCode): String = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 9135753041f92..18649a39cbc9e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -91,7 +91,7 @@ case class DateAdd(startDate: Expression, days: Expression) start.asInstanceOf[Int] + d.asInstanceOf[Int] } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (sd, d) => { s"""${ev.value} = $sd + $d;""" }) @@ -119,7 +119,7 @@ case class DateSub(startDate: Expression, days: Expression) start.asInstanceOf[Int] - d.asInstanceOf[Int] } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (sd, d) => { s"""${ev.value} = $sd - $d;""" }) @@ -141,7 +141,7 @@ case class Hour(child: Expression) extends UnaryExpression with ImplicitCastInpu DateTimeUtils.getHours(timestamp.asInstanceOf[Long]) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, c => s"$dtu.getHours($c)") } @@ -160,7 +160,7 @@ case class Minute(child: Expression) extends UnaryExpression with ImplicitCastIn DateTimeUtils.getMinutes(timestamp.asInstanceOf[Long]) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, c => s"$dtu.getMinutes($c)") } @@ -179,7 +179,7 @@ case class Second(child: Expression) extends UnaryExpression with ImplicitCastIn DateTimeUtils.getSeconds(timestamp.asInstanceOf[Long]) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, c => s"$dtu.getSeconds($c)") } @@ -198,7 +198,7 @@ case class DayOfYear(child: Expression) extends UnaryExpression with ImplicitCas DateTimeUtils.getDayInYear(date.asInstanceOf[Int]) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, c => s"$dtu.getDayInYear($c)") } @@ -217,7 +217,7 @@ case class Year(child: Expression) extends UnaryExpression with ImplicitCastInpu DateTimeUtils.getYear(date.asInstanceOf[Int]) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, c => s"$dtu.getYear($c)") } @@ -235,7 +235,7 @@ case class Quarter(child: Expression) extends UnaryExpression with ImplicitCastI DateTimeUtils.getQuarter(date.asInstanceOf[Int]) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, c => s"$dtu.getQuarter($c)") } @@ -254,7 +254,7 @@ case class Month(child: Expression) extends UnaryExpression with ImplicitCastInp DateTimeUtils.getMonth(date.asInstanceOf[Int]) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, c => s"$dtu.getMonth($c)") } @@ -273,7 +273,7 @@ case class DayOfMonth(child: Expression) extends UnaryExpression with ImplicitCa DateTimeUtils.getDayOfMonth(date.asInstanceOf[Int]) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, c => s"$dtu.getDayOfMonth($c)") } @@ -300,7 +300,7 @@ case class WeekOfYear(child: Expression) extends UnaryExpression with ImplicitCa c.get(Calendar.WEEK_OF_YEAR) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, time => { val cal = classOf[Calendar].getName val c = ctx.freshName("cal") @@ -335,7 +335,7 @@ case class DateFormatClass(left: Expression, right: Expression) extends BinaryEx UTF8String.fromString(sdf.format(new java.util.Date(timestamp.asInstanceOf[Long] / 1000))) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val sdf = classOf[SimpleDateFormat].getName defineCodeGen(ctx, ev, (timestamp, format) => { s"""UTF8String.fromString((new $sdf($format.toString())) @@ -430,7 +430,7 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { left.dataType match { case StringType if right.foldable => val sdf = classOf[SimpleDateFormat].getName @@ -442,7 +442,7 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; """ } else { - val eval1 = left.gen(ctx) + val eval1 = left.genCode(ctx) s""" ${eval1.code} boolean ${ev.isNull} = ${eval1.isNull}; @@ -471,7 +471,7 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { """ }) case TimestampType => - val eval1 = left.gen(ctx) + val eval1 = left.genCode(ctx) s""" ${eval1.code} boolean ${ev.isNull} = ${eval1.isNull}; @@ -482,7 +482,7 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { """ case DateType => val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - val eval1 = left.gen(ctx) + val eval1 = left.genCode(ctx) s""" ${eval1.code} boolean ${ev.isNull} = ${eval1.isNull}; @@ -550,7 +550,7 @@ case class FromUnixTime(sec: Expression, format: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val sdf = classOf[SimpleDateFormat].getName if (format.foldable) { if (constFormat == null) { @@ -559,7 +559,7 @@ case class FromUnixTime(sec: Expression, format: Expression) ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; """ } else { - val t = left.gen(ctx) + val t = left.genCode(ctx) s""" ${t.code} boolean ${ev.isNull} = ${t.isNull}; @@ -605,7 +605,7 @@ case class LastDay(startDate: Expression) extends UnaryExpression with ImplicitC DateTimeUtils.getLastDayOfMonth(date.asInstanceOf[Int]) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, sd => s"$dtu.getLastDayOfMonth($sd)") } @@ -646,7 +646,7 @@ case class NextDay(startDate: Expression, dayOfWeek: Expression) } } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (sd, dowS) => { val dateTimeUtilClass = DateTimeUtils.getClass.getName.stripSuffix("$") val dayOfWeekTerm = ctx.freshName("dayOfWeek") @@ -698,7 +698,7 @@ case class TimeAdd(start: Expression, interval: Expression) start.asInstanceOf[Long], itvl.months, itvl.microseconds) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (sd, i) => { s"""$dtu.timestampAddInterval($sd, $i.months, $i.microseconds)""" @@ -725,7 +725,7 @@ case class FromUTCTimestamp(left: Expression, right: Expression) timezone.asInstanceOf[UTF8String].toString) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") if (right.foldable) { val tz = right.eval() @@ -738,7 +738,7 @@ case class FromUTCTimestamp(left: Expression, right: Expression) val tzTerm = ctx.freshName("tz") val tzClass = classOf[TimeZone].getName ctx.addMutableState(tzClass, tzTerm, s"""$tzTerm = $tzClass.getTimeZone("$tz");""") - val eval = left.gen(ctx) + val eval = left.genCode(ctx) s""" |${eval.code} |boolean ${ev.isNull} = ${eval.isNull}; @@ -777,7 +777,7 @@ case class TimeSub(start: Expression, interval: Expression) start.asInstanceOf[Long], 0 - itvl.months, 0 - itvl.microseconds) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (sd, i) => { s"""$dtu.timestampAddInterval($sd, 0 - $i.months, 0 - $i.microseconds)""" @@ -805,7 +805,7 @@ case class AddMonths(startDate: Expression, numMonths: Expression) DateTimeUtils.dateAddMonths(start.asInstanceOf[Int], months.asInstanceOf[Int]) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (sd, m) => { s"""$dtu.dateAddMonths($sd, $m)""" @@ -835,7 +835,7 @@ case class MonthsBetween(date1: Expression, date2: Expression) DateTimeUtils.monthsBetween(t1.asInstanceOf[Long], t2.asInstanceOf[Long]) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (l, r) => { s"""$dtu.monthsBetween($l, $r)""" @@ -864,7 +864,7 @@ case class ToUTCTimestamp(left: Expression, right: Expression) timezone.asInstanceOf[UTF8String].toString) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") if (right.foldable) { val tz = right.eval() @@ -877,7 +877,7 @@ case class ToUTCTimestamp(left: Expression, right: Expression) val tzTerm = ctx.freshName("tz") val tzClass = classOf[TimeZone].getName ctx.addMutableState(tzClass, tzTerm, s"""$tzTerm = $tzClass.getTimeZone("$tz");""") - val eval = left.gen(ctx) + val eval = left.genCode(ctx) s""" |${eval.code} |boolean ${ev.isNull} = ${eval.isNull}; @@ -912,7 +912,7 @@ case class ToDate(child: Expression) extends UnaryExpression with ImplicitCastIn override def eval(input: InternalRow): Any = child.eval(input) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, d => d) } @@ -959,7 +959,7 @@ case class TruncDate(date: Expression, format: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") if (format.foldable) { @@ -969,7 +969,7 @@ case class TruncDate(date: Expression, format: Expression) ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; """ } else { - val d = date.gen(ctx) + val d = date.genCode(ctx) s""" ${d.code} boolean ${ev.isNull} = ${d.isNull}; @@ -1013,7 +1013,7 @@ case class DateDiff(endDate: Expression, startDate: Expression) end.asInstanceOf[Int] - start.asInstanceOf[Int] } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (end, start) => s"$end - $start") } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala index 74e86f40c0364..5629ee1a1489b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala @@ -34,7 +34,7 @@ case class UnscaledValue(child: Expression) extends UnaryExpression { protected override def nullSafeEval(input: Any): Any = input.asInstanceOf[Decimal].toUnscaledLong - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"$c.toUnscaledLong()") } } @@ -53,7 +53,7 @@ case class MakeDecimal(child: Expression, precision: Int, scale: Int) extends Un protected override def nullSafeEval(input: Any): Any = Decimal(input.asInstanceOf[Long], precision, scale) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, eval => { s""" ${ev.value} = (new Decimal()).setOrNull($eval, $precision, $scale); @@ -70,8 +70,8 @@ case class MakeDecimal(child: Expression, precision: Int, scale: Int) extends Un case class PromotePrecision(child: Expression) extends UnaryExpression { override def dataType: DataType = child.dataType override def eval(input: InternalRow): Any = child.eval(input) - override def gen(ctx: CodegenContext): ExprCode = child.gen(ctx) - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = "" + override def genCode(ctx: CodegenContext): ExprCode = child.genCode(ctx) + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = "" override def prettyName: String = "promote_precision" override def sql: String = child.sql } @@ -93,7 +93,7 @@ case class CheckOverflow(child: Expression, dataType: DecimalType) extends Unary } } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, eval => { val tmp = ctx.freshName("tmp") s""" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 7fd4bc3066cbe..bdadbfbbb0f2f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -191,7 +191,7 @@ case class Literal protected (value: Any, dataType: DataType) override def eval(input: InternalRow): Any = value - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { // change the isNull and primitive to consts, to inline them if (value == null) { ev.isNull = "true" @@ -205,7 +205,7 @@ case class Literal protected (value: Any, dataType: DataType) case FloatType => val v = value.asInstanceOf[Float] if (v.isNaN || v.isInfinite) { - super[CodegenFallback].genCode(ctx, ev) + super[CodegenFallback].doGenCode(ctx, ev) } else { ev.isNull = "false" ev.value = s"${value}f" @@ -214,7 +214,7 @@ case class Literal protected (value: Any, dataType: DataType) case DoubleType => val v = value.asInstanceOf[Double] if (v.isNaN || v.isInfinite) { - super[CodegenFallback].genCode(ctx, ev) + super[CodegenFallback].doGenCode(ctx, ev) } else { ev.isNull = "false" ev.value = s"${value}D" @@ -234,7 +234,7 @@ case class Literal protected (value: Any, dataType: DataType) "" // eval() version may be faster for non-primitive types case other => - super[CodegenFallback].genCode(ctx, ev) + super[CodegenFallback].doGenCode(ctx, ev) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala index 9e190289b7db0..231382e6bb503 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala @@ -70,7 +70,7 @@ abstract class UnaryMathExpression(val f: Double => Double, name: String) // name of function in java.lang.Math def funcName: String = name.toLowerCase - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"java.lang.Math.${funcName}($c)") } } @@ -88,7 +88,7 @@ abstract class UnaryLogExpression(f: Double => Double, name: String) if (d <= yAsymptote) null else f(d) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, c => s""" if ($c <= $yAsymptote) { @@ -123,7 +123,7 @@ abstract class BinaryMathExpression(f: (Double, Double) => Double, name: String) f(input1.asInstanceOf[Double], input2.asInstanceOf[Double]) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (c1, c2) => s"java.lang.Math.${name.toLowerCase}($c1, $c2)") } } @@ -197,7 +197,7 @@ case class Ceil(child: Expression) extends UnaryMathExpression(math.ceil, "CEIL" case DecimalType.Fixed(precision, scale) => input.asInstanceOf[Decimal].ceil } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { child.dataType match { case DecimalType.Fixed(_, 0) => defineCodeGen(ctx, ev, c => s"$c") case DecimalType.Fixed(precision, scale) => @@ -242,7 +242,7 @@ case class Conv(numExpr: Expression, fromBaseExpr: Expression, toBaseExpr: Expre toBase.asInstanceOf[Int]) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val numconv = NumberConverter.getClass.getName.stripSuffix("$") nullSafeCodeGen(ctx, ev, (num, from, to) => s""" @@ -284,7 +284,7 @@ case class Floor(child: Expression) extends UnaryMathExpression(math.floor, "FLO case DecimalType.Fixed(precision, scale) => input.asInstanceOf[Decimal].floor } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { child.dataType match { case DecimalType.Fixed(_, 0) => defineCodeGen(ctx, ev, c => s"$c") case DecimalType.Fixed(precision, scale) => @@ -346,7 +346,7 @@ case class Factorial(child: Expression) extends UnaryExpression with ImplicitCas } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, eval => { s""" if ($eval > 20 || $eval < 0) { @@ -370,7 +370,7 @@ case class Log(child: Expression) extends UnaryLogExpression(math.log, "LOG") extended = "> SELECT _FUNC_(2);\n 1.0") case class Log2(child: Expression) extends UnaryLogExpression((x: Double) => math.log(x) / math.log(2), "LOG2") { - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, c => s""" if ($c <= $yAsymptote) { @@ -458,7 +458,7 @@ case class Bin(child: Expression) protected override def nullSafeEval(input: Any): Any = UTF8String.fromString(jl.Long.toBinaryString(input.asInstanceOf[Long])) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (c) => s"UTF8String.fromString(java.lang.Long.toBinaryString($c))") } @@ -556,7 +556,7 @@ case class Hex(child: Expression) extends UnaryExpression with ImplicitCastInput case StringType => Hex.hex(num.asInstanceOf[UTF8String].getBytes) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (c) => { val hex = Hex.getClass.getName.stripSuffix("$") s"${ev.value} = " + (child.dataType match { @@ -584,7 +584,7 @@ case class Unhex(child: Expression) extends UnaryExpression with ImplicitCastInp protected override def nullSafeEval(num: Any): Any = Hex.unhex(num.asInstanceOf[UTF8String].getBytes) - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (c) => { val hex = Hex.getClass.getName.stripSuffix("$") s""" @@ -613,7 +613,7 @@ case class Atan2(left: Expression, right: Expression) math.atan2(input1.asInstanceOf[Double] + 0.0, input2.asInstanceOf[Double] + 0.0) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (c1, c2) => s"java.lang.Math.atan2($c1 + 0.0, $c2 + 0.0)") } } @@ -623,7 +623,7 @@ case class Atan2(left: Expression, right: Expression) extended = "> SELECT _FUNC_(2, 3);\n 8.0") case class Pow(left: Expression, right: Expression) extends BinaryMathExpression(math.pow, "POWER") { - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (c1, c2) => s"java.lang.Math.pow($c1, $c2)") } } @@ -653,7 +653,7 @@ case class ShiftLeft(left: Expression, right: Expression) } } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (left, right) => s"$left << $right") } } @@ -683,7 +683,7 @@ case class ShiftRight(left: Expression, right: Expression) } } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (left, right) => s"$left >> $right") } } @@ -713,7 +713,7 @@ case class ShiftRightUnsigned(left: Expression, right: Expression) } } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (left, right) => s"$left >>> $right") } } @@ -753,7 +753,7 @@ case class Logarithm(left: Expression, right: Expression) if (dLeft <= 0.0 || dRight <= 0.0) null else math.log(dRight) / math.log(dLeft) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { if (left.isInstanceOf[EulerNumber]) { nullSafeCodeGen(ctx, ev, (c1, c2) => s""" @@ -874,8 +874,8 @@ abstract class RoundBase(child: Expression, scale: Expression, } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val ce = child.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val ce = child.genCode(ctx) val evaluationCode = child.dataType match { case _: DecimalType => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index f2f0c2d698efa..8bef2524cce80 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -49,7 +49,7 @@ case class Md5(child: Expression) extends UnaryExpression with ImplicitCastInput protected override def nullSafeEval(input: Any): Any = UTF8String.fromString(DigestUtils.md5Hex(input.asInstanceOf[Array[Byte]])) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"UTF8String.fromString(org.apache.commons.codec.digest.DigestUtils.md5Hex($c))") } @@ -102,7 +102,7 @@ case class Sha2(left: Expression, right: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val digestUtils = "org.apache.commons.codec.digest.DigestUtils" nullSafeCodeGen(ctx, ev, (eval1, eval2) => { s""" @@ -147,7 +147,7 @@ case class Sha1(child: Expression) extends UnaryExpression with ImplicitCastInpu protected override def nullSafeEval(input: Any): Any = UTF8String.fromString(DigestUtils.sha1Hex(input.asInstanceOf[Array[Byte]])) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"UTF8String.fromString(org.apache.commons.codec.digest.DigestUtils.sha1Hex($c))" ) @@ -173,7 +173,7 @@ case class Crc32(child: Expression) extends UnaryExpression with ImplicitCastInp checksum.getValue } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val CRC32 = "java.util.zip.CRC32" nullSafeCodeGen(ctx, ev, value => { s""" @@ -244,10 +244,10 @@ abstract class HashExpression[E] extends Expression { protected def computeHash(value: Any, dataType: DataType, seed: E): E - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { ev.isNull = "false" val childrenHash = children.map { child => - val childGen = child.gen(ctx) + val childGen = child.genCode(ctx) childGen.code + ctx.nullSafeExec(child.nullable, childGen.isNull) { computeHash(childGen.value, child.dataType, ev.value, ctx) } @@ -477,7 +477,7 @@ case class PrintToStderr(child: Expression) extends UnaryExpression { protected override def nullSafeEval(input: Any): Any = input - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, c => s""" | System.err.println("Result of ${child.simpleString} is " + $c); @@ -510,8 +510,8 @@ case class AssertTrue(child: Expression) extends UnaryExpression with ImplicitCa } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval = child.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval = child.genCode(ctx) ev.isNull = "true" ev.value = "null" s"""${eval.code} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 78310fb2f1539..b0434674c651f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -142,8 +142,8 @@ case class Alias(child: Expression, name: String)( override def eval(input: InternalRow): Any = child.eval(input) /** Just a simple passthrough for code generation. */ - override def gen(ctx: CodegenContext): ExprCode = child.gen(ctx) - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = "" + override def genCode(ctx: CodegenContext): ExprCode = child.genCode(ctx) + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = "" override def dataType: DataType = child.dataType override def nullable: Boolean = child.nullable diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala index 6a452499430c8..d9c06e3b99ca3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala @@ -64,17 +64,17 @@ case class Coalesce(children: Seq[Expression]) extends Expression { result } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val first = children(0) val rest = children.drop(1) - val firstEval = first.gen(ctx) + val firstEval = first.genCode(ctx) s""" ${firstEval.code} boolean ${ev.isNull} = ${firstEval.isNull}; ${ctx.javaType(dataType)} ${ev.value} = ${firstEval.value}; """ + rest.map { e => - val eval = e.gen(ctx) + val eval = e.genCode(ctx) s""" if (${ev.isNull}) { ${eval.code} @@ -113,8 +113,8 @@ case class IsNaN(child: Expression) extends UnaryExpression } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval = child.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval = child.genCode(ctx) child.dataType match { case DoubleType | FloatType => s""" @@ -155,9 +155,9 @@ case class NaNvl(left: Expression, right: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val leftGen = left.gen(ctx) - val rightGen = right.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val leftGen = left.genCode(ctx) + val rightGen = right.genCode(ctx) left.dataType match { case DoubleType | FloatType => s""" @@ -196,8 +196,8 @@ case class IsNull(child: Expression) extends UnaryExpression with Predicate { child.eval(input) == null } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval = child.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval = child.genCode(ctx) ev.isNull = "false" ev.value = eval.isNull eval.code @@ -219,8 +219,8 @@ case class IsNotNull(child: Expression) extends UnaryExpression with Predicate { child.eval(input) != null } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval = child.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval = child.genCode(ctx) ev.isNull = "false" ev.value = s"(!(${eval.isNull}))" eval.code @@ -259,10 +259,10 @@ case class AtLeastNNonNulls(n: Int, children: Seq[Expression]) extends Predicate numNonNulls >= n } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val nonnull = ctx.freshName("nonnull") val code = children.map { e => - val eval = e.gen(ctx) + val eval = e.genCode(ctx) e.dataType match { case DoubleType | FloatType => s""" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala index 26b1ff39b3e9f..f5f102a578f2b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala @@ -59,9 +59,9 @@ case class StaticInvoke( override def eval(input: InternalRow): Any = throw new UnsupportedOperationException("Only code-generated evaluation is supported.") - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val javaType = ctx.javaType(dataType) - val argGen = arguments.map(_.gen(ctx)) + val argGen = arguments.map(_.genCode(ctx)) val argString = argGen.map(_.value).mkString(", ") if (propagateNull) { @@ -148,10 +148,10 @@ case class Invoke( case _ => identity[String] _ } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val javaType = ctx.javaType(dataType) - val obj = targetObject.gen(ctx) - val argGen = arguments.map(_.gen(ctx)) + val obj = targetObject.genCode(ctx) + val argGen = arguments.map(_.genCode(ctx)) val argString = argGen.map(_.value).mkString(", ") // If the function can return null, we do an extra check to make sure our null bit is still set @@ -239,12 +239,12 @@ case class NewInstance( override def eval(input: InternalRow): Any = throw new UnsupportedOperationException("Only code-generated evaluation is supported.") - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val javaType = ctx.javaType(dataType) - val argGen = arguments.map(_.gen(ctx)) + val argGen = arguments.map(_.genCode(ctx)) val argString = argGen.map(_.value).mkString(", ") - val outer = outerPointer.map(func => Literal.fromObject(func()).gen(ctx)) + val outer = outerPointer.map(func => Literal.fromObject(func()).genCode(ctx)) val setup = s""" @@ -302,9 +302,9 @@ case class UnwrapOption( override def eval(input: InternalRow): Any = throw new UnsupportedOperationException("Only code-generated evaluation is supported") - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val javaType = ctx.javaType(dataType) - val inputObject = child.gen(ctx) + val inputObject = child.genCode(ctx) s""" ${inputObject.code} @@ -319,6 +319,7 @@ case class UnwrapOption( /** * Converts the result of evaluating `child` into an option, checking both the isNull bit and * (in the case of reference types) equality with null. + * * @param child The expression to evaluate and wrap. * @param optType The type of this option. */ @@ -334,8 +335,8 @@ case class WrapOption(child: Expression, optType: DataType) override def eval(input: InternalRow): Any = throw new UnsupportedOperationException("Only code-generated evaluation is supported") - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val inputObject = child.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val inputObject = child.genCode(ctx) s""" ${inputObject.code} @@ -357,7 +358,7 @@ case class LambdaVariable(value: String, isNull: String, dataType: DataType) ext override def nullable: Boolean = true - override def gen(ctx: CodegenContext): ExprCode = { + override def genCode(ctx: CodegenContext): ExprCode = { ExprCode(code = "", value = value, isNull = isNull) } } @@ -443,13 +444,13 @@ case class MapObjects private( override def dataType: DataType = ArrayType(lambdaFunction.dataType) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val javaType = ctx.javaType(dataType) val elementJavaType = ctx.javaType(loopVar.dataType) ctx.addMutableState("boolean", loopVar.isNull, "") ctx.addMutableState(elementJavaType, loopVar.value, "") - val genInputData = inputData.gen(ctx) - val genFunction = lambdaFunction.gen(ctx) + val genInputData = inputData.genCode(ctx) + val genFunction = lambdaFunction.genCode(ctx) val dataLength = ctx.freshName("dataLength") val convertedArray = ctx.freshName("convertedArray") val loopIndex = ctx.freshName("loopIndex") @@ -523,13 +524,13 @@ case class CreateExternalRow(children: Seq[Expression], schema: StructType) override def eval(input: InternalRow): Any = throw new UnsupportedOperationException("Only code-generated evaluation is supported") - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val rowClass = classOf[GenericRowWithSchema].getName val values = ctx.freshName("values") ctx.addMutableState("Object[]", values, "") val childrenCodes = children.zipWithIndex.map { case (e, i) => - val eval = e.gen(ctx) + val eval = e.genCode(ctx) eval.code + s""" if (${eval.isNull}) { $values[$i] = null; @@ -551,6 +552,7 @@ case class CreateExternalRow(children: Seq[Expression], schema: StructType) /** * Serializes an input object using a generic serializer (Kryo or Java). + * * @param kryo if true, use Kryo. Otherwise, use Java. */ case class EncodeUsingSerializer(child: Expression, kryo: Boolean) @@ -559,7 +561,7 @@ case class EncodeUsingSerializer(child: Expression, kryo: Boolean) override def eval(input: InternalRow): Any = throw new UnsupportedOperationException("Only code-generated evaluation is supported") - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { // Code to initialize the serializer. val serializer = ctx.freshName("serializer") val (serializerClass, serializerInstanceClass) = { @@ -576,7 +578,7 @@ case class EncodeUsingSerializer(child: Expression, kryo: Boolean) s"$serializer = ($serializerInstanceClass) new $serializerClass($sparkConf).newInstance();") // Code to serialize. - val input = child.gen(ctx) + val input = child.genCode(ctx) s""" ${input.code} final boolean ${ev.isNull} = ${input.isNull}; @@ -593,12 +595,13 @@ case class EncodeUsingSerializer(child: Expression, kryo: Boolean) /** * Serializes an input object using a generic serializer (Kryo or Java). Note that the ClassTag * is not an implicit parameter because TreeNode cannot copy implicit parameters. + * * @param kryo if true, use Kryo. Otherwise, use Java. */ case class DecodeUsingSerializer[T](child: Expression, tag: ClassTag[T], kryo: Boolean) extends UnaryExpression with NonSQLExpression { - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { // Code to initialize the serializer. val serializer = ctx.freshName("serializer") val (serializerClass, serializerInstanceClass) = { @@ -615,7 +618,7 @@ case class DecodeUsingSerializer[T](child: Expression, tag: ClassTag[T], kryo: B s"$serializer = ($serializerInstanceClass) new $serializerClass($sparkConf).newInstance();") // Code to serialize. - val input = child.gen(ctx) + val input = child.genCode(ctx) s""" ${input.code} final boolean ${ev.isNull} = ${input.isNull}; @@ -643,12 +646,12 @@ case class InitializeJavaBean(beanInstance: Expression, setters: Map[String, Exp override def eval(input: InternalRow): Any = throw new UnsupportedOperationException("Only code-generated evaluation is supported.") - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val instanceGen = beanInstance.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val instanceGen = beanInstance.genCode(ctx) val initialize = setters.map { case (setterMethod, fieldValue) => - val fieldGen = fieldValue.gen(ctx) + val fieldGen = fieldValue.genCode(ctx) s""" ${fieldGen.code} ${instanceGen.value}.$setterMethod(${fieldGen.value}); @@ -685,8 +688,8 @@ case class AssertNotNull(child: Expression, walkedTypePath: Seq[String]) override def eval(input: InternalRow): Any = throw new UnsupportedOperationException("Only code-generated evaluation is supported.") - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val childGen = child.gen(ctx) + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val childGen = child.genCode(ctx) val errMsg = "Null value appeared in non-nullable field:" + walkedTypePath.mkString("\n", "\n", "\n") + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 38f1210a4edb5..b15a77a8e7b29 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -99,7 +99,7 @@ case class Not(child: Expression) protected override def nullSafeEval(input: Any): Any = !input.asInstanceOf[Boolean] - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"!($c)") } @@ -157,9 +157,9 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val valueGen = value.gen(ctx) - val listGen = list.map(_.gen(ctx)) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val valueGen = value.genCode(ctx) + val listGen = list.map(_.genCode(ctx)) val listCode = listGen.map(x => s""" if (!${ev.value}) { @@ -216,10 +216,10 @@ case class InSet(child: Expression, hset: Set[Any]) extends UnaryExpression with def getHSet(): Set[Any] = hset - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val setName = classOf[Set[Any]].getName val InSetName = classOf[InSet].getName - val childGen = child.gen(ctx) + val childGen = child.genCode(ctx) ctx.references += this val hsetTerm = ctx.freshName("hset") val hasNullTerm = ctx.freshName("hasNull") @@ -274,9 +274,9 @@ case class And(left: Expression, right: Expression) extends BinaryOperator with } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval1 = left.gen(ctx) - val eval2 = right.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval1 = left.genCode(ctx) + val eval2 = right.genCode(ctx) // The result should be `false`, if any of them is `false` whenever the other is null or not. if (!left.nullable && !right.nullable) { @@ -339,9 +339,9 @@ case class Or(left: Expression, right: Expression) extends BinaryOperator with P } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval1 = left.gen(ctx) - val eval2 = right.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval1 = left.genCode(ctx) + val eval2 = right.genCode(ctx) // The result should be `true`, if any of them is `true` whenever the other is null or not. if (!left.nullable && !right.nullable) { @@ -379,7 +379,7 @@ case class Or(left: Expression, right: Expression) extends BinaryOperator with P abstract class BinaryComparison extends BinaryOperator with Predicate { - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { if (ctx.isPrimitiveType(left.dataType) && left.dataType != BooleanType // java boolean doesn't support > or < operator && left.dataType != FloatType @@ -428,7 +428,7 @@ case class EqualTo(left: Expression, right: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (c1, c2) => ctx.genEqual(left.dataType, c1, c2)) } } @@ -464,9 +464,9 @@ case class EqualNullSafe(left: Expression, right: Expression) extends BinaryComp } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val eval1 = left.gen(ctx) - val eval2 = right.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val eval1 = left.genCode(ctx) + val eval2 = right.genCode(ctx) val equalCode = ctx.genEqual(left.dataType, eval1.value, eval2.value) ev.isNull = "false" eval1.code + eval2.code + s""" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala index 1ec092a5be965..1eed24dd1e5b5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala @@ -67,7 +67,7 @@ case class Rand(seed: Long) extends RDG { case _ => throw new AnalysisException("Input argument to rand must be an integer literal.") }) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val rngTerm = ctx.freshName("rng") val className = classOf[XORShiftRandom].getName ctx.addMutableState(className, rngTerm, @@ -92,7 +92,7 @@ case class Randn(seed: Long) extends RDG { case _ => throw new AnalysisException("Input argument to randn must be an integer literal.") }) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val rngTerm = ctx.freshName("rng") val className = classOf[XORShiftRandom].getName ctx.addMutableState(className, rngTerm, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala index 85a54292639d0..4f5b85d7f432c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala @@ -78,7 +78,7 @@ case class Like(left: Expression, right: Expression) override def toString: String = s"$left LIKE $right" - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val patternClass = classOf[Pattern].getName val escapeFunc = StringUtils.getClass.getName.stripSuffix("$") + ".escapeLikeRegex" val pattern = ctx.freshName("pattern") @@ -92,7 +92,7 @@ case class Like(left: Expression, right: Expression) s"""$pattern = ${patternClass}.compile("$regexStr");""") // We don't use nullSafeCodeGen here because we don't want to re-evaluate right again. - val eval = left.gen(ctx) + val eval = left.genCode(ctx) s""" ${eval.code} boolean ${ev.isNull} = ${eval.isNull}; @@ -128,7 +128,7 @@ case class RLike(left: Expression, right: Expression) override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).find(0) override def toString: String = s"$left RLIKE $right" - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val patternClass = classOf[Pattern].getName val pattern = ctx.freshName("pattern") @@ -141,7 +141,7 @@ case class RLike(left: Expression, right: Expression) s"""$pattern = ${patternClass}.compile("$regexStr");""") // We don't use nullSafeCodeGen here because we don't want to re-evaluate right again. - val eval = left.gen(ctx) + val eval = left.genCode(ctx) s""" ${eval.code} boolean ${ev.isNull} = ${eval.isNull}; @@ -188,7 +188,7 @@ case class StringSplit(str: Expression, pattern: Expression) new GenericArrayData(strings.asInstanceOf[Array[Any]]) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val arrayClass = classOf[GenericArrayData].getName nullSafeCodeGen(ctx, ev, (str, pattern) => // Array in java is covariant, so we don't need to cast UTF8String[] to Object[]. @@ -247,7 +247,7 @@ case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expressio override def children: Seq[Expression] = subject :: regexp :: rep :: Nil override def prettyName: String = "regexp_replace" - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val termLastRegex = ctx.freshName("lastRegex") val termPattern = ctx.freshName("pattern") @@ -330,7 +330,7 @@ case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expressio override def children: Seq[Expression] = subject :: regexp :: idx :: Nil override def prettyName: String = "regexp_extract" - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val termLastRegex = ctx.freshName("lastRegex") val termPattern = ctx.freshName("pattern") val classNamePattern = classOf[Pattern].getCanonicalName diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index a17482697d906..8c15357360413 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -51,8 +51,8 @@ case class Concat(children: Seq[Expression]) extends Expression with ImplicitCas UTF8String.concat(inputs : _*) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val evals = children.map(_.gen(ctx)) + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val evals = children.map(_.genCode(ctx)) val inputs = evals.map { eval => s"${eval.isNull} ? null : ${eval.value}" }.mkString(", ") @@ -106,10 +106,10 @@ case class ConcatWs(children: Seq[Expression]) UTF8String.concatWs(flatInputs.head, flatInputs.tail : _*) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { if (children.forall(_.dataType == StringType)) { // All children are strings. In that case we can construct a fixed size array. - val evals = children.map(_.gen(ctx)) + val evals = children.map(_.genCode(ctx)) val inputs = evals.map { eval => s"${eval.isNull} ? (UTF8String) null : ${eval.value}" @@ -124,7 +124,7 @@ case class ConcatWs(children: Seq[Expression]) val varargNum = ctx.freshName("varargNum") val idxInVararg = ctx.freshName("idxInVararg") - val evals = children.map(_.gen(ctx)) + val evals = children.map(_.genCode(ctx)) val (varargCount, varargBuild) = children.tail.zip(evals.tail).map { case (child, eval) => child.dataType match { case StringType => @@ -185,7 +185,7 @@ case class Upper(child: Expression) override def convert(v: UTF8String): UTF8String = v.toUpperCase - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"($c).toUpperCase()") } } @@ -200,7 +200,7 @@ case class Lower(child: Expression) extends UnaryExpression with String2StringEx override def convert(v: UTF8String): UTF8String = v.toLowerCase - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"($c).toLowerCase()") } } @@ -225,7 +225,7 @@ trait StringPredicate extends Predicate with ImplicitCastInputTypes { case class Contains(left: Expression, right: Expression) extends BinaryExpression with StringPredicate { override def compare(l: UTF8String, r: UTF8String): Boolean = l.contains(r) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (c1, c2) => s"($c1).contains($c2)") } } @@ -236,7 +236,7 @@ case class Contains(left: Expression, right: Expression) case class StartsWith(left: Expression, right: Expression) extends BinaryExpression with StringPredicate { override def compare(l: UTF8String, r: UTF8String): Boolean = l.startsWith(r) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (c1, c2) => s"($c1).startsWith($c2)") } } @@ -247,7 +247,7 @@ case class StartsWith(left: Expression, right: Expression) case class EndsWith(left: Expression, right: Expression) extends BinaryExpression with StringPredicate { override def compare(l: UTF8String, r: UTF8String): Boolean = l.endsWith(r) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (c1, c2) => s"($c1).endsWith($c2)") } } @@ -298,7 +298,7 @@ case class StringTranslate(srcExpr: Expression, matchingExpr: Expression, replac srcEval.asInstanceOf[UTF8String].translate(dict) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { val termLastMatching = ctx.freshName("lastMatching") val termLastReplace = ctx.freshName("lastReplace") val termDict = ctx.freshName("dict") @@ -351,7 +351,7 @@ case class FindInSet(left: Expression, right: Expression) extends BinaryExpressi override protected def nullSafeEval(word: Any, set: Any): Any = set.asInstanceOf[UTF8String].findInSet(word.asInstanceOf[UTF8String]) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (word, set) => s"${ev.value} = $set.findInSet($word);" ) @@ -375,7 +375,7 @@ case class StringTrim(child: Expression) override def prettyName: String = "trim" - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"($c).trim()") } } @@ -393,7 +393,7 @@ case class StringTrimLeft(child: Expression) override def prettyName: String = "ltrim" - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"($c).trimLeft()") } } @@ -411,7 +411,7 @@ case class StringTrimRight(child: Expression) override def prettyName: String = "rtrim" - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"($c).trimRight()") } } @@ -440,7 +440,7 @@ case class StringInstr(str: Expression, substr: Expression) override def prettyName: String = "instr" - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (l, r) => s"($l).indexOf($r, 0) + 1") } @@ -475,7 +475,7 @@ case class SubstringIndex(strExpr: Expression, delimExpr: Expression, countExpr: count.asInstanceOf[Int]) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (str, delim, count) => s"$str.subStringIndex($delim, $count)") } } @@ -524,10 +524,10 @@ case class StringLocate(substr: Expression, str: Expression, start: Expression) } } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val substrGen = substr.gen(ctx) - val strGen = str.gen(ctx) - val startGen = start.gen(ctx) + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val substrGen = substr.genCode(ctx) + val strGen = str.genCode(ctx) + val startGen = start.genCode(ctx) s""" int ${ev.value} = 0; boolean ${ev.isNull} = false; @@ -571,7 +571,7 @@ case class StringLPad(str: Expression, len: Expression, pad: Expression) str.asInstanceOf[UTF8String].lpad(len.asInstanceOf[Int], pad.asInstanceOf[UTF8String]) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (str, len, pad) => s"$str.lpad($len, $pad)") } @@ -597,7 +597,7 @@ case class StringRPad(str: Expression, len: Expression, pad: Expression) str.asInstanceOf[UTF8String].rpad(len.asInstanceOf[Int], pad.asInstanceOf[UTF8String]) } - override protected def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (str, len, pad) => s"$str.rpad($len, $pad)") } @@ -638,10 +638,10 @@ case class FormatString(children: Expression*) extends Expression with ImplicitC } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - val pattern = children.head.gen(ctx) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + val pattern = children.head.genCode(ctx) - val argListGen = children.tail.map(x => (x.dataType, x.gen(ctx))) + val argListGen = children.tail.map(x => (x.dataType, x.genCode(ctx))) val argListCode = argListGen.map(_._2.code + "\n") val argListString = argListGen.foldLeft("")((s, v) => { @@ -694,7 +694,7 @@ case class InitCap(child: Expression) extends UnaryExpression with ImplicitCastI override def nullSafeEval(string: Any): Any = { string.asInstanceOf[UTF8String].toLowerCase.toTitleCase } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, str => s"$str.toLowerCase().toTitleCase()") } } @@ -719,7 +719,7 @@ case class StringRepeat(str: Expression, times: Expression) override def prettyName: String = "repeat" - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (l, r) => s"($l).repeat($r)") } } @@ -735,7 +735,7 @@ case class StringReverse(child: Expression) extends UnaryExpression with String2 override def prettyName: String = "reverse" - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"($c).reverse()") } } @@ -757,7 +757,7 @@ case class StringSpace(child: Expression) UTF8String.blankString(if (length < 0) 0 else length) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (length) => s"""${ev.value} = UTF8String.blankString(($length < 0) ? 0 : $length);""") } @@ -799,7 +799,7 @@ case class Substring(str: Expression, pos: Expression, len: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, (string, pos, len) => { str.dataType match { @@ -825,7 +825,7 @@ case class Length(child: Expression) extends UnaryExpression with ExpectsInputTy case BinaryType => value.asInstanceOf[Array[Byte]].length } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { child.dataType match { case StringType => defineCodeGen(ctx, ev, c => s"($c).numChars()") case BinaryType => defineCodeGen(ctx, ev, c => s"($c).length") @@ -848,7 +848,7 @@ case class Levenshtein(left: Expression, right: Expression) extends BinaryExpres protected override def nullSafeEval(leftValue: Any, rightValue: Any): Any = leftValue.asInstanceOf[UTF8String].levenshteinDistance(rightValue.asInstanceOf[UTF8String]) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (left, right) => s"${ev.value} = $left.levenshteinDistance($right);") } @@ -868,7 +868,7 @@ case class SoundEx(child: Expression) extends UnaryExpression with ExpectsInputT override def nullSafeEval(input: Any): Any = input.asInstanceOf[UTF8String].soundex() - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { defineCodeGen(ctx, ev, c => s"$c.soundex()") } } @@ -894,7 +894,7 @@ case class Ascii(child: Expression) extends UnaryExpression with ImplicitCastInp } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (child) => { val bytes = ctx.freshName("bytes") s""" @@ -924,7 +924,7 @@ case class Base64(child: Expression) extends UnaryExpression with ImplicitCastIn bytes.asInstanceOf[Array[Byte]])) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (child) => { s"""${ev.value} = UTF8String.fromBytes( org.apache.commons.codec.binary.Base64.encodeBase64($child)); @@ -945,7 +945,7 @@ case class UnBase64(child: Expression) extends UnaryExpression with ImplicitCast protected override def nullSafeEval(string: Any): Any = org.apache.commons.codec.binary.Base64.decodeBase64(string.asInstanceOf[UTF8String].toString) - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (child) => { s""" ${ev.value} = org.apache.commons.codec.binary.Base64.decodeBase64($child.toString()); @@ -973,7 +973,7 @@ case class Decode(bin: Expression, charset: Expression) UTF8String.fromString(new String(input1.asInstanceOf[Array[Byte]], fromCharset)) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (bytes, charset) => s""" try { @@ -1005,7 +1005,7 @@ case class Encode(value: Expression, charset: Expression) input1.asInstanceOf[UTF8String].toString.getBytes(toCharset) } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (string, charset) => s""" try { @@ -1088,7 +1088,7 @@ case class FormatNumber(x: Expression, d: Expression) } } - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { nullSafeCodeGen(ctx, ev, (num, d) => { def typeHelper(p: String): String = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NonFoldableLiteral.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NonFoldableLiteral.scala index ff34b1e37be93..de410b86ea4db 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NonFoldableLiteral.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NonFoldableLiteral.scala @@ -35,8 +35,8 @@ case class NonFoldableLiteral(value: Any, dataType: DataType) extends LeafExpres override def eval(input: InternalRow): Any = value - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - Literal.create(value, dataType).genCode(ctx, ev) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + Literal.create(value, dataType).doGenCode(ctx, ev) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 392c48fb7b93b..3dc2aa33dfd7c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -192,7 +192,7 @@ private[sql] case class RowDataSourceScan( val row = ctx.freshName("row") ctx.INPUT_ROW = row ctx.currentVars = null - val columnsRowInput = exprRows.map(_.gen(ctx)) + val columnsRowInput = exprRows.map(_.genCode(ctx)) val inputRow = if (outputUnsafeRows) row else null s""" |while ($input.hasNext()) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala index bd23b7e3ad683..cc0382e5d4bd3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala @@ -149,7 +149,7 @@ case class Expand( val firstExpr = projections.head(col) if (sameOutput(col)) { // This column is the same across all output rows. Just generate code for it here. - BindReferences.bindReference(firstExpr, child.output).gen(ctx) + BindReferences.bindReference(firstExpr, child.output).genCode(ctx) } else { val isNull = ctx.freshName("isNull") val value = ctx.freshName("value") @@ -166,7 +166,7 @@ case class Expand( var updateCode = "" for (col <- exprs.indices) { if (!sameOutput(col)) { - val ev = BindReferences.bindReference(exprs(col), child.output).gen(ctx) + val ev = BindReferences.bindReference(exprs(col), child.output).genCode(ctx) updateCode += s""" |${ev.code} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala index 29acc38ab3584..12d08c8c45e67 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala @@ -118,7 +118,7 @@ trait CodegenSupport extends SparkPlan { ctx.currentVars = null ctx.INPUT_ROW = row output.zipWithIndex.map { case (attr, i) => - BoundReference(i, attr.dataType, attr.nullable).gen(ctx) + BoundReference(i, attr.dataType, attr.nullable).genCode(ctx) } } else { assert(outputVars != null) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala index f585759e583c0..d819a6599390e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala @@ -165,7 +165,7 @@ case class TungstenAggregate( ctx.addMutableState("boolean", isNull, "") ctx.addMutableState(ctx.javaType(e.dataType), value, "") // The initial expression should not access any column - val ev = e.gen(ctx) + val ev = e.genCode(ctx) val initVars = s""" | $isNull = ${ev.isNull}; | $value = ${ev.value}; @@ -179,13 +179,13 @@ case class TungstenAggregate( // evaluate aggregate results ctx.currentVars = bufVars val aggResults = functions.map(_.evaluateExpression).map { e => - BindReferences.bindReference(e, aggregateBufferAttributes).gen(ctx) + BindReferences.bindReference(e, aggregateBufferAttributes).genCode(ctx) } val evaluateAggResults = evaluateVariables(aggResults) // evaluate result expressions ctx.currentVars = aggResults val resultVars = resultExpressions.map { e => - BindReferences.bindReference(e, aggregateAttributes).gen(ctx) + BindReferences.bindReference(e, aggregateAttributes).genCode(ctx) } (resultVars, s""" |$evaluateAggResults @@ -196,7 +196,7 @@ case class TungstenAggregate( (bufVars, "") } else { // no aggregate function, the result should be literals - val resultVars = resultExpressions.map(_.gen(ctx)) + val resultVars = resultExpressions.map(_.genCode(ctx)) (resultVars, evaluateVariables(resultVars)) } @@ -240,7 +240,7 @@ case class TungstenAggregate( } ctx.currentVars = bufVars ++ input // TODO: support subexpression elimination - val aggVals = updateExpr.map(BindReferences.bindReference(_, inputAttrs).gen(ctx)) + val aggVals = updateExpr.map(BindReferences.bindReference(_, inputAttrs).genCode(ctx)) // aggregate buffer should be updated atomic val updates = aggVals.zipWithIndex.map { case (ev, i) => s""" @@ -394,25 +394,25 @@ case class TungstenAggregate( ctx.currentVars = null ctx.INPUT_ROW = keyTerm val keyVars = groupingExpressions.zipWithIndex.map { case (e, i) => - BoundReference(i, e.dataType, e.nullable).gen(ctx) + BoundReference(i, e.dataType, e.nullable).genCode(ctx) } val evaluateKeyVars = evaluateVariables(keyVars) ctx.INPUT_ROW = bufferTerm val bufferVars = aggregateBufferAttributes.zipWithIndex.map { case (e, i) => - BoundReference(i, e.dataType, e.nullable).gen(ctx) + BoundReference(i, e.dataType, e.nullable).genCode(ctx) } val evaluateBufferVars = evaluateVariables(bufferVars) // evaluate the aggregation result ctx.currentVars = bufferVars val aggResults = declFunctions.map(_.evaluateExpression).map { e => - BindReferences.bindReference(e, aggregateBufferAttributes).gen(ctx) + BindReferences.bindReference(e, aggregateBufferAttributes).genCode(ctx) } val evaluateAggResults = evaluateVariables(aggResults) // generate the final result ctx.currentVars = keyVars ++ aggResults val inputAttrs = groupingAttributes ++ aggregateAttributes val resultVars = resultExpressions.map { e => - BindReferences.bindReference(e, inputAttrs).gen(ctx) + BindReferences.bindReference(e, inputAttrs).genCode(ctx) } s""" $evaluateKeyVars @@ -437,7 +437,7 @@ case class TungstenAggregate( ctx.INPUT_ROW = keyTerm ctx.currentVars = null val eval = resultExpressions.map{ e => - BindReferences.bindReference(e, groupingAttributes).gen(ctx) + BindReferences.bindReference(e, groupingAttributes).genCode(ctx) } consume(ctx, eval) } @@ -576,7 +576,7 @@ case class TungstenAggregate( // generate hash code for key val hashExpr = Murmur3Hash(groupingExpressions, 42) ctx.currentVars = input - val hashEval = BindReferences.bindReference(hashExpr, child.output).gen(ctx) + val hashEval = BindReferences.bindReference(hashExpr, child.output).genCode(ctx) val inputAttr = aggregateBufferAttributes ++ child.output ctx.currentVars = new Array[ExprCode](aggregateBufferAttributes.length) ++ input @@ -613,7 +613,8 @@ case class TungstenAggregate( val updateRowInVectorizedHashMap: Option[String] = { if (isVectorizedHashMapEnabled) { ctx.INPUT_ROW = vectorizedRowBuffer - val vectorizedRowEvals = updateExpr.map(BindReferences.bindReference(_, inputAttr).gen(ctx)) + val vectorizedRowEvals = + updateExpr.map(BindReferences.bindReference(_, inputAttr).genCode(ctx)) val updateVectorizedRow = vectorizedRowEvals.zipWithIndex.map { case (ev, i) => val dt = updateExpr(i).dataType ctx.updateColumn(vectorizedRowBuffer, dt, i, ev, updateExpr(i).nullable) @@ -663,7 +664,7 @@ case class TungstenAggregate( val updateRowInUnsafeRowMap: String = { ctx.INPUT_ROW = unsafeRowBuffer val unsafeRowBufferEvals = - updateExpr.map(BindReferences.bindReference(_, inputAttr).gen(ctx)) + updateExpr.map(BindReferences.bindReference(_, inputAttr).genCode(ctx)) val updateUnsafeRowBuffer = unsafeRowBufferEvals.zipWithIndex.map { case (ev, i) => val dt = updateExpr(i).dataType ctx.updateColumn(unsafeRowBuffer, dt, i, ev, updateExpr(i).nullable) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 344aaff348e77..c689fc3fbb263 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -53,7 +53,7 @@ case class Project(projectList: Seq[NamedExpression], child: SparkPlan) val exprs = projectList.map(x => ExpressionCanonicalizer.execute(BindReferences.bindReference(x, child.output))) ctx.currentVars = input - val resultVars = exprs.map(_.gen(ctx)) + val resultVars = exprs.map(_.genCode(ctx)) // Evaluation of non-deterministic expressions can't be deferred. val nonDeterministicAttrs = projectList.filterNot(_.deterministic).map(_.toAttribute) s""" @@ -122,7 +122,7 @@ case class Filter(condition: Expression, child: SparkPlan) val evaluated = evaluateRequiredVariables(child.output, in, c.references) // Generate the code for the predicate. - val ev = ExpressionCanonicalizer.execute(bound).gen(ctx) + val ev = ExpressionCanonicalizer.execute(bound).genCode(ctx) val nullCheck = if (bound.nullable) { s"${ev.isNull} || " } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala index a8f854136c1f9..b94b0d26b2f26 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala @@ -118,7 +118,7 @@ case class BroadcastHashJoin( ctx.currentVars = input if (streamedKeys.length == 1 && streamedKeys.head.dataType == LongType) { // generate the join key as Long - val ev = streamedKeys.head.gen(ctx) + val ev = streamedKeys.head.genCode(ctx) (ev, ev.isNull) } else { // generate the join key as UnsafeRow @@ -134,7 +134,7 @@ case class BroadcastHashJoin( ctx.currentVars = null ctx.INPUT_ROW = matched buildPlan.output.zipWithIndex.map { case (a, i) => - val ev = BoundReference(i, a.dataType, a.nullable).gen(ctx) + val ev = BoundReference(i, a.dataType, a.nullable).genCode(ctx) if (joinType == Inner) { ev } else { @@ -170,7 +170,8 @@ case class BroadcastHashJoin( val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references) // filter the output via condition ctx.currentVars = input ++ buildVars - val ev = BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).gen(ctx) + val ev = + BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx) s""" |$eval |${ev.code} @@ -244,7 +245,8 @@ case class BroadcastHashJoin( // evaluate the variables from build side that used by condition val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references) ctx.currentVars = input ++ buildVars - val ev = BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).gen(ctx) + val ev = + BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx) s""" |boolean $conditionPassed = true; |${eval.trim} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala index 0e7b2f2f3187f..443a7b43b6e9a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala @@ -226,7 +226,7 @@ case class SortMergeJoin( keys: Seq[Expression], input: Seq[Attribute]): Seq[ExprCode] = { ctx.INPUT_ROW = row - keys.map(BindReferences.bindReference(_, input).gen(ctx)) + keys.map(BindReferences.bindReference(_, input).genCode(ctx)) } private def copyKeys(ctx: CodegenContext, vars: Seq[ExprCode]): Seq[ExprCode] = { @@ -376,7 +376,7 @@ case class SortMergeJoin( private def createRightVar(ctx: CodegenContext, rightRow: String): Seq[ExprCode] = { ctx.INPUT_ROW = rightRow right.output.zipWithIndex.map { case (a, i) => - BoundReference(i, a.dataType, a.nullable).gen(ctx) + BoundReference(i, a.dataType, a.nullable).genCode(ctx) } } @@ -427,7 +427,7 @@ case class SortMergeJoin( val (rightBefore, rightAfter) = splitVarsByCondition(right.output, rightVars) // Generate code for condition ctx.currentVars = leftVars ++ rightVars - val cond = BindReferences.bindReference(condition.get, output).gen(ctx) + val cond = BindReferences.bindReference(condition.get, output).genCode(ctx) // evaluate the columns those used by condition before loop val before = s""" |boolean $loaded = false; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index d2ab18ef0e189..784b1e8c26fe1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -48,7 +48,7 @@ case class DeserializeToObject( val bound = ExpressionCanonicalizer.execute( BindReferences.bindReference(deserializer, child.output)) ctx.currentVars = input - val resultVars = bound.gen(ctx) :: Nil + val resultVars = bound.genCode(ctx) :: Nil consume(ctx, resultVars) } @@ -82,7 +82,7 @@ case class SerializeFromObject( ExpressionCanonicalizer.execute(BindReferences.bindReference(expr, child.output)) } ctx.currentVars = input - val resultVars = bound.map(_.gen(ctx)) + val resultVars = bound.map(_.genCode(ctx)) consume(ctx, resultVars) } @@ -173,13 +173,13 @@ case class MapElements( val bound = ExpressionCanonicalizer.execute( BindReferences.bindReference(callFunc, child.output)) ctx.currentVars = input - val evaluated = bound.gen(ctx) + val evaluated = bound.genCode(ctx) val resultObj = LambdaVariable(evaluated.value, evaluated.isNull, resultObjType) val outputFields = serializer.map(_ transform { case _: BoundReference => resultObj }) - val resultVars = outputFields.map(_.gen(ctx)) + val resultVars = outputFields.map(_.genCode(ctx)) s""" ${evaluated.code} ${consume(ctx, resultVars)} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 4b3091ba22c60..03defc121cf09 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -54,8 +54,8 @@ case class ScalarSubquery( override def eval(input: InternalRow): Any = result - override def genCode(ctx: CodegenContext, ev: ExprCode): String = { - Literal.create(result, dataType).genCode(ctx, ev) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): String = { + Literal.create(result, dataType).doGenCode(ctx, ev) } } From f1a11976db5cd7a4f2c5467ef784b6755f84260c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 18 Apr 2016 14:28:47 -0700 Subject: [PATCH 13/69] [SPARK-14674][SQL] Move HiveContext.hiveconf to HiveSessionState ## What changes were proposed in this pull request? This is just cleanup. This allows us to remove HiveContext later without inflating the diff too much. This PR fixes the conflicts of https://github.com/apache/spark/pull/12431. It also removes the `def hiveConf` from `HiveSqlParser`. So, we will pass the HiveConf associated with a session explicitly instead of relying on Hive's `SessionState` to pass `HiveConf`. ## How was this patch tested? Existing tests. Closes #12431 Author: Andrew Or Author: Yin Huai Closes #12449 from yhuai/hiveconf. --- .../hive/thriftserver/HiveThriftServer2.scala | 4 ++-- .../sql/hive/thriftserver/SparkSQLEnv.scala | 5 ++--- .../apache/spark/sql/hive/HiveContext.scala | 19 ++-------------- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../spark/sql/hive/HiveSessionCatalog.scala | 2 +- .../spark/sql/hive/HiveSessionState.scala | 18 ++++++++++++++- .../apache/spark/sql/hive/TableReader.scala | 8 ++++--- .../sql/hive/execution/HiveSqlParser.scala | 22 +++---------------- .../sql/hive/execution/HiveTableScan.scala | 2 +- .../hive/execution/InsertIntoHiveTable.scala | 18 +++++++-------- .../hive/execution/ScriptTransformation.scala | 2 +- .../spark/sql/hive/execution/commands.scala | 3 ++- .../apache/spark/sql/hive/test/TestHive.scala | 10 ++++----- .../spark/sql/hive/ErrorPositionSuite.scala | 2 +- .../spark/sql/hive/HiveDDLCommandSuite.scala | 5 +++-- .../spark/sql/hive/SerializationSuite.scala | 2 +- .../spark/sql/hive/StatisticsSuite.scala | 2 +- .../sql/hive/execution/HiveQuerySuite.scala | 4 ++-- 18 files changed, 59 insertions(+), 71 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index ee0d23a6e57c4..6703cdbac3d17 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -55,7 +55,7 @@ object HiveThriftServer2 extends Logging { @DeveloperApi def startWithContext(sqlContext: HiveContext): Unit = { val server = new HiveThriftServer2(sqlContext) - server.init(sqlContext.hiveconf) + server.init(sqlContext.sessionState.hiveconf) server.start() listener = new HiveThriftServer2Listener(server, sqlContext.conf) sqlContext.sparkContext.addSparkListener(listener) @@ -83,7 +83,7 @@ object HiveThriftServer2 extends Logging { try { val server = new HiveThriftServer2(SparkSQLEnv.hiveContext) - server.init(SparkSQLEnv.hiveContext.hiveconf) + server.init(SparkSQLEnv.hiveContext.sessionState.hiveconf) server.start() logInfo("HiveThriftServer2 started") listener = new HiveThriftServer2Listener(server, SparkSQLEnv.hiveContext.conf) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 2594c5bfdb3af..ae1d737b58adc 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -65,9 +65,8 @@ private[hive] object SparkSQLEnv extends Logging { hiveContext.setConf("spark.sql.hive.version", HiveContext.hiveExecutionVersion) if (log.isDebugEnabled) { - hiveContext.hiveconf.getAllProperties.asScala.toSeq.sorted.foreach { case (k, v) => - logDebug(s"HiveConf var: $k=$v") - } + hiveContext.sessionState.hiveconf.getAllProperties.asScala.toSeq.sorted + .foreach { case (k, v) => logDebug(s"HiveConf var: $k=$v") } } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 71ef99a6a9952..b26a9ab699be1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -153,7 +153,7 @@ class HiveContext private[hive]( protected[sql] override def parseSql(sql: String): LogicalPlan = { executionHive.withHiveState { - super.parseSql(substitutor.substitute(hiveconf, sql)) + super.parseSql(substitutor.substitute(sessionState.hiveconf, sql)) } } @@ -200,28 +200,13 @@ class HiveContext private[hive]( // Also, calling hiveconf will create a default session containing a HiveConf, which // will interfer with the creation of executionHive (which is a lazy val). So, // we put hiveconf.set at the end of this method. - hiveconf.set(key, value) + sessionState.hiveconf.set(key, value) } override private[sql] def setConf[T](entry: ConfigEntry[T], value: T): Unit = { setConf(entry.key, entry.stringConverter(value)) } - /** - * SQLConf and HiveConf contracts: - * - * 1. create a new o.a.h.hive.ql.session.SessionState for each HiveContext - * 2. when the Hive session is first initialized, params in HiveConf will get picked up by the - * SQLConf. Additionally, any properties set by set() or a SET command inside sql() will be - * set in the SQLConf *as well as* in the HiveConf. - */ - @transient - protected[hive] lazy val hiveconf: HiveConf = { - val c = executionHive.conf - setConf(c.getAllProperties) - c - } - private def functionOrMacroDDLPattern(command: String) = Pattern.compile( ".*(create|drop)\\s+(temporary\\s+)?(function|macro).+", Pattern.DOTALL).matcher(command) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index ccc8345d7375d..697cf719c164e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -299,7 +299,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte CatalogTableType.MANAGED_TABLE } - val maybeSerDe = HiveSerDe.sourceToSerDe(provider, hive.hiveconf) + val maybeSerDe = HiveSerDe.sourceToSerDe(provider, hive.sessionState.hiveconf) val dataSource = DataSource( hive, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index 0cccc22e5a624..f91393fc76324 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -75,7 +75,7 @@ private[sql] class HiveSessionCatalog( // ---------------------------------------------------------------- override def getDefaultDBPath(db: String): String = { - val defaultPath = context.hiveconf.getVar(HiveConf.ConfVars.METASTOREWAREHOUSE) + val defaultPath = context.sessionState.hiveconf.getVar(HiveConf.ConfVars.METASTOREWAREHOUSE) new Path(new Path(defaultPath), db + ".db").toString } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala index bc28b55d06d9e..2b848524f3e90 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive +import org.apache.hadoop.hive.conf.HiveConf + import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.Analyzer import org.apache.spark.sql.catalyst.parser.ParserInterface @@ -32,6 +34,20 @@ import org.apache.spark.sql.internal.{SessionState, SQLConf} */ private[hive] class HiveSessionState(ctx: HiveContext) extends SessionState(ctx) { + /** + * SQLConf and HiveConf contracts: + * + * 1. create a new o.a.h.hive.ql.session.SessionState for each [[HiveContext]] + * 2. when the Hive session is first initialized, params in HiveConf will get picked up by the + * SQLConf. Additionally, any properties set by set() or a SET command inside sql() will be + * set in the SQLConf *as well as* in the HiveConf. + */ + lazy val hiveconf: HiveConf = { + val c = ctx.executionHive.conf + ctx.setConf(c.getAllProperties) + c + } + /** * A Hive client used for execution. */ @@ -80,7 +96,7 @@ private[hive] class HiveSessionState(ctx: HiveContext) extends SessionState(ctx) /** * Parser for HiveQl query texts. */ - override lazy val sqlParser: ParserInterface = HiveSqlParser + override lazy val sqlParser: ParserInterface = new HiveSqlParser(hiveconf) /** * Planner that takes into account Hive-specific strategies. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index 54afe9c2a3550..98a427380d97b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -72,7 +72,9 @@ class HadoopTableReader( private val _minSplitsPerRDD = if (sc.sparkContext.isLocal) { 0 // will splitted based on block by default. } else { - math.max(sc.hiveconf.getInt("mapred.map.tasks", 1), sc.sparkContext.defaultMinPartitions) + math.max( + sc.sessionState.hiveconf.getInt("mapred.map.tasks", 1), + sc.sparkContext.defaultMinPartitions) } SparkHadoopUtil.get.appendS3AndSparkHadoopConfigurations(sc.sparkContext.conf, hiveExtraConf) @@ -162,7 +164,7 @@ class HadoopTableReader( case (partition, partDeserializer) => def updateExistPathSetByPathPattern(pathPatternStr: String) { val pathPattern = new Path(pathPatternStr) - val fs = pathPattern.getFileSystem(sc.hiveconf) + val fs = pathPattern.getFileSystem(sc.sessionState.hiveconf) val matches = fs.globStatus(pathPattern) matches.foreach(fileStatus => existPathSet += fileStatus.getPath.toString) } @@ -259,7 +261,7 @@ class HadoopTableReader( private def applyFilterIfNeeded(path: Path, filterOpt: Option[PathFilter]): String = { filterOpt match { case Some(filter) => - val fs = path.getFileSystem(sc.hiveconf) + val fs = path.getFileSystem(sc.sessionState.hiveconf) val filteredFiles = fs.listStatus(path, filter).map(_.getPath.toString) filteredFiles.mkString(",") case None => path.toString diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala index a97b65e27bc59..d5d3ee43d7e82 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala @@ -39,8 +39,8 @@ import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper /** * Concrete parser for HiveQl statements. */ -object HiveSqlParser extends AbstractSqlParser { - val astBuilder = new HiveSqlAstBuilder +class HiveSqlParser(hiveConf: HiveConf) extends AbstractSqlParser { + val astBuilder = new HiveSqlAstBuilder(hiveConf) override protected def nativeCommand(sqlText: String): LogicalPlan = { HiveNativeCommand(sqlText) @@ -50,25 +50,9 @@ object HiveSqlParser extends AbstractSqlParser { /** * Builder that converts an ANTLR ParseTree into a LogicalPlan/Expression/TableIdentifier. */ -class HiveSqlAstBuilder extends SparkSqlAstBuilder { +class HiveSqlAstBuilder(hiveConf: HiveConf) extends SparkSqlAstBuilder { import ParserUtils._ - /** - * Get the current Hive Configuration. - */ - private[this] def hiveConf: HiveConf = { - var ss = SessionState.get() - // SessionState is lazy initialization, it can be null here - if (ss == null) { - val original = Thread.currentThread().getContextClassLoader - val conf = new HiveConf(classOf[SessionState]) - conf.setClassLoader(original) - ss = new SessionState(conf) - SessionState.start(ss) - } - ss.getConf - } - /** * Pass a command to Hive using a [[HiveNativeCommand]]. */ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala index 235b80b7c697c..3c46b836dcec3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala @@ -75,7 +75,7 @@ case class HiveTableScan( // Create a local copy of hiveconf,so that scan specific modifications should not impact // other queries @transient - private[this] val hiveExtraConf = new HiveConf(context.hiveconf) + private[this] val hiveExtraConf = new HiveConf(context.sessionState.hiveconf) // append columns ids and names before broadcast addColumnMetadataToConf(hiveExtraConf) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 430fa4616fc2b..ed538630d24a8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -44,7 +44,7 @@ case class InsertIntoHiveTable( ifNotExists: Boolean) extends UnaryNode { @transient val sc: HiveContext = sqlContext.asInstanceOf[HiveContext] - @transient private lazy val hiveContext = new Context(sc.hiveconf) + @transient private lazy val hiveContext = new Context(sc.sessionState.hiveconf) @transient private lazy val client = sc.metadataHive def output: Seq[Attribute] = Seq.empty @@ -86,17 +86,17 @@ case class InsertIntoHiveTable( val tableLocation = table.hiveQlTable.getDataLocation val tmpLocation = hiveContext.getExternalTmpPath(tableLocation) val fileSinkConf = new FileSinkDesc(tmpLocation.toString, tableDesc, false) - val isCompressed = sc.hiveconf.getBoolean( + val isCompressed = sc.sessionState.hiveconf.getBoolean( ConfVars.COMPRESSRESULT.varname, ConfVars.COMPRESSRESULT.defaultBoolVal) if (isCompressed) { // Please note that isCompressed, "mapred.output.compress", "mapred.output.compression.codec", // and "mapred.output.compression.type" have no impact on ORC because it uses table properties // to store compression information. - sc.hiveconf.set("mapred.output.compress", "true") + sc.sessionState.hiveconf.set("mapred.output.compress", "true") fileSinkConf.setCompressed(true) - fileSinkConf.setCompressCodec(sc.hiveconf.get("mapred.output.compression.codec")) - fileSinkConf.setCompressType(sc.hiveconf.get("mapred.output.compression.type")) + fileSinkConf.setCompressCodec(sc.sessionState.hiveconf.get("mapred.output.compression.codec")) + fileSinkConf.setCompressType(sc.sessionState.hiveconf.get("mapred.output.compression.type")) } val numDynamicPartitions = partition.values.count(_.isEmpty) @@ -113,13 +113,13 @@ case class InsertIntoHiveTable( // Validate partition spec if there exist any dynamic partitions if (numDynamicPartitions > 0) { // Report error if dynamic partitioning is not enabled - if (!sc.hiveconf.getBoolVar(HiveConf.ConfVars.DYNAMICPARTITIONING)) { + if (!sc.sessionState.hiveconf.getBoolVar(HiveConf.ConfVars.DYNAMICPARTITIONING)) { throw new SparkException(ErrorMsg.DYNAMIC_PARTITION_DISABLED.getMsg) } // Report error if dynamic partition strict mode is on but no static partition is found - if (numStaticPartitions == 0 && - sc.hiveconf.getVar(HiveConf.ConfVars.DYNAMICPARTITIONINGMODE).equalsIgnoreCase("strict")) { + if (numStaticPartitions == 0 && sc.sessionState.hiveconf.getVar( + HiveConf.ConfVars.DYNAMICPARTITIONINGMODE).equalsIgnoreCase("strict")) { throw new SparkException(ErrorMsg.DYNAMIC_PARTITION_STRICT_MODE.getMsg) } @@ -130,7 +130,7 @@ case class InsertIntoHiveTable( } } - val jobConf = new JobConf(sc.hiveconf) + val jobConf = new JobConf(sc.sessionState.hiveconf) val jobConfSer = new SerializableJobConf(jobConf) // When speculation is on and output committer class name contains "Direct", we should warn diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index 3566526561b2f..ea48b0e5c26de 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -64,7 +64,7 @@ case class ScriptTransformation( override def producedAttributes: AttributeSet = outputSet -- inputSet - private val serializedHiveConf = new SerializableConfiguration(sc.hiveconf) + private val serializedHiveConf = new SerializableConfiguration(sc.sessionState.hiveconf) protected override def doExecute(): RDD[InternalRow] = { def processIterator(inputIterator: Iterator[InternalRow]): Iterator[InternalRow] = { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 0c06608ff9903..5ef502afa5e67 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -105,7 +105,8 @@ case class AnalyzeTable(tableName: String) extends RunnableCommand { Option(tableParameters.get(StatsSetupConst.TOTAL_SIZE)) .map(_.toLong) .getOrElse(0L) - val newTotalSize = getFileSizeForTable(hiveContext.hiveconf, relation.hiveQlTable) + val newTotalSize = + getFileSizeForTable(hiveContext.sessionState.hiveconf, relation.hiveQlTable) // Update the Hive metastore if the total size of the table is different than the size // recorded in the Hive metastore. // This logic is based on org.apache.hadoop.hive.ql.exec.StatsTask.aggregateStats(). diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index d56d36fe32e77..2767528395d9d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -112,9 +112,9 @@ class TestHiveContext private[hive]( // By clearing the port we force Spark to pick a new one. This allows us to rerun tests // without restarting the JVM. System.clearProperty("spark.hostPort") - CommandProcessorFactory.clean(hiveconf) + CommandProcessorFactory.clean(sessionState.hiveconf) - hiveconf.set("hive.plan.serialization.format", "javaXML") + sessionState.hiveconf.set("hive.plan.serialization.format", "javaXML") // A snapshot of the entries in the starting SQLConf // We save this because tests can mutate this singleton object if they want @@ -136,7 +136,7 @@ class TestHiveContext private[hive]( // Override so we can intercept relative paths and rewrite them to point at hive. override def runSqlHive(sql: String): Seq[String] = - super.runSqlHive(rewritePaths(substitutor.substitute(this.hiveconf, sql))) + super.runSqlHive(rewritePaths(substitutor.substitute(sessionState.hiveconf, sql))) override def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution(plan) @@ -461,7 +461,7 @@ class TestHiveContext private[hive]( foreach { udfName => FunctionRegistry.unregisterTemporaryUDF(udfName) } // Some tests corrupt this value on purpose, which breaks the RESET call below. - hiveconf.set("fs.default.name", new File(".").toURI.toString) + sessionState.hiveconf.set("fs.default.name", new File(".").toURI.toString) // It is important that we RESET first as broken hooks that might have been set could break // other sql exec here. executionHive.runSqlHive("RESET") @@ -476,7 +476,7 @@ class TestHiveContext private[hive]( // In case a test changed any of these values, restore all the original ones here. TestHiveContext.hiveClientConfigurations( - hiveconf, warehousePath, scratchDirPath, metastoreTemporaryConf) + sessionState.hiveconf, warehousePath, scratchDirPath, metastoreTemporaryConf) .foreach { case (k, v) => metadataHive.runSqlHive(s"SET $k=$v") } defaultOverrides() diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala index d9664680f4a11..b9e7a36b41a1a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala @@ -131,7 +131,7 @@ class ErrorPositionSuite extends QueryTest with TestHiveSingleton with BeforeAnd * @param token a unique token in the string that should be indicated by the exception */ def positionTest(name: String, query: String, token: String): Unit = { - def ast = HiveSqlParser.parsePlan(query) + def ast = hiveContext.sessionState.sqlParser.parsePlan(query) def parseTree = Try(quietly(ast.treeString)).getOrElse("") test(name) { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala index 110c6d19d89ba..484cf528e6db7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDDLCommandSuite.scala @@ -30,10 +30,11 @@ import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.{Generate, ScriptTransformation} import org.apache.spark.sql.execution.command.{CreateTable, CreateTableLike} -import org.apache.spark.sql.hive.execution.{HiveNativeCommand, HiveSqlParser} +import org.apache.spark.sql.hive.execution.HiveNativeCommand +import org.apache.spark.sql.hive.test.TestHive class HiveDDLCommandSuite extends PlanTest { - val parser = HiveSqlParser + val parser = TestHive.sessionState.sqlParser private def extractTableDesc(sql: String): (CatalogTable, Boolean) = { parser.parsePlan(sql).collect { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala index 93dcb10f7a296..ac3a65032fb0a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/SerializationSuite.scala @@ -24,7 +24,7 @@ class SerializationSuite extends SparkFunSuite { test("[SPARK-5840] HiveContext should be serializable") { val hiveContext = org.apache.spark.sql.hive.test.TestHive - hiveContext.hiveconf + hiveContext.sessionState.hiveconf val serializer = new JavaSerializer(new SparkConf()).newInstance() val bytes = serializer.serialize(hiveContext) val deSer = serializer.deserialize[AnyRef](bytes) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 05318f51af01e..d14c72b34bc9a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -31,7 +31,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { test("parse analyze commands") { def assertAnalyzeCommand(analyzeCommand: String, c: Class[_]) { - val parsed = HiveSqlParser.parsePlan(analyzeCommand) + val parsed = hiveContext.sessionState.sqlParser.parsePlan(analyzeCommand) val operators = parsed.collect { case a: AnalyzeTable => a case o => o diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index af73baa1f3914..2e7a1d921b75c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -1157,11 +1157,11 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { collectResults(sql(s"SET $testKey=$testVal")) } - assert(hiveconf.get(testKey, "") === testVal) + assert(sessionState.hiveconf.get(testKey, "") === testVal) assertResult(defaults ++ Set(testKey -> testVal))(collectResults(sql("SET"))) sql(s"SET ${testKey + testKey}=${testVal + testVal}") - assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) + assert(sessionState.hiveconf.get(testKey + testKey, "") == testVal + testVal) assertResult(defaults ++ Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { collectResults(sql("SET")) } From 68450c8c6e34df510f9e131147210a6c0c2eb56b Mon Sep 17 00:00:00 2001 From: Luciano Resende Date: Mon, 18 Apr 2016 14:35:10 -0700 Subject: [PATCH 14/69] [SPARK-14504][SQL] Enable Oracle docker tests ## What changes were proposed in this pull request? Enable Oracle docker tests ## How was this patch tested? Existing tests Author: Luciano Resende Closes #12270 from lresende/oracle. --- external/docker-integration-tests/pom.xml | 21 +++++++++++-------- .../sql/jdbc/OracleIntegrationSuite.scala | 2 +- project/SparkBuild.scala | 4 ++-- 3 files changed, 15 insertions(+), 12 deletions(-) diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 17fd7d781c9ab..53a24f3e06e08 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -36,8 +36,12 @@ - db2 + db https://app.camunda.com/nexus/content/repositories/public/ + + true + warn + @@ -143,14 +147,13 @@ to use a an ojdbc jar for the testcase. The maven dependency here is commented because currently the maven repository does not contain the ojdbc jar mentioned. Once the jar is available in maven, this could be uncommented. --> - + + com.oracle + ojdbc6 + 11.2.0.1.0 + test + + upstreams() -------> upstreams() ------> execute() + * doExecute() ---------> inputRDDs() -------> inputRDDs() ------> execute() * | * +-----------------> produce() * | @@ -350,8 +350,8 @@ case class WholeStageCodegen(child: SparkPlan) extends UnaryNode with CodegenSup val durationMs = longMetric("pipelineTime") - val rdds = child.asInstanceOf[CodegenSupport].upstreams() - assert(rdds.size <= 2, "Up to two upstream RDDs can be supported") + val rdds = child.asInstanceOf[CodegenSupport].inputRDDs() + assert(rdds.size <= 2, "Up to two input RDDs can be supported") if (rdds.length == 1) { rdds.head.mapPartitionsWithIndex { (index, iter) => val clazz = CodeGenerator.compile(cleanedSource) @@ -367,7 +367,7 @@ case class WholeStageCodegen(child: SparkPlan) extends UnaryNode with CodegenSup } } } else { - // Right now, we support up to two upstreams. + // Right now, we support up to two input RDDs. rdds.head.zipPartitions(rdds(1)) { (leftIter, rightIter) => val partitionIndex = TaskContext.getPartitionId() val clazz = CodeGenerator.compile(cleanedSource) @@ -385,7 +385,7 @@ case class WholeStageCodegen(child: SparkPlan) extends UnaryNode with CodegenSup } } - override def upstreams(): Seq[RDD[InternalRow]] = { + override def inputRDDs(): Seq[RDD[InternalRow]] = { throw new UnsupportedOperationException } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala index d819a6599390e..89977f9e086b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala @@ -129,8 +129,8 @@ case class TungstenAggregate( !aggregateExpressions.exists(_.aggregateFunction.isInstanceOf[ImperativeAggregate]) } - override def upstreams(): Seq[RDD[InternalRow]] = { - child.asInstanceOf[CodegenSupport].upstreams() + override def inputRDDs(): Seq[RDD[InternalRow]] = { + child.asInstanceOf[CodegenSupport].inputRDDs() } protected override def doProduce(ctx: CodegenContext): String = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index c689fc3fbb263..892c57ae7d7c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -31,8 +31,8 @@ case class Project(projectList: Seq[NamedExpression], child: SparkPlan) override def output: Seq[Attribute] = projectList.map(_.toAttribute) - override def upstreams(): Seq[RDD[InternalRow]] = { - child.asInstanceOf[CodegenSupport].upstreams() + override def inputRDDs(): Seq[RDD[InternalRow]] = { + child.asInstanceOf[CodegenSupport].inputRDDs() } protected override def doProduce(ctx: CodegenContext): String = { @@ -103,8 +103,8 @@ case class Filter(condition: Expression, child: SparkPlan) private[sql] override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) - override def upstreams(): Seq[RDD[InternalRow]] = { - child.asInstanceOf[CodegenSupport].upstreams() + override def inputRDDs(): Seq[RDD[InternalRow]] = { + child.asInstanceOf[CodegenSupport].inputRDDs() } protected override def doProduce(ctx: CodegenContext): String = { @@ -243,8 +243,8 @@ case class Sample( } } - override def upstreams(): Seq[RDD[InternalRow]] = { - child.asInstanceOf[CodegenSupport].upstreams() + override def inputRDDs(): Seq[RDD[InternalRow]] = { + child.asInstanceOf[CodegenSupport].inputRDDs() } protected override def doProduce(ctx: CodegenContext): String = { @@ -315,7 +315,7 @@ case class Range( // output attributes should not affect the results override lazy val cleanArgs: Seq[Any] = Seq(start, step, numSlices, numElements) - override def upstreams(): Seq[RDD[InternalRow]] = { + override def inputRDDs(): Seq[RDD[InternalRow]] = { sqlContext.sparkContext.parallelize(0 until numSlices, numSlices) .map(i => InternalRow(i)) :: Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index 17eae88b49dec..e6079ecaadc7d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -164,8 +164,8 @@ package object debug { } } - override def upstreams(): Seq[RDD[InternalRow]] = { - child.asInstanceOf[CodegenSupport].upstreams() + override def inputRDDs(): Seq[RDD[InternalRow]] = { + child.asInstanceOf[CodegenSupport].inputRDDs() } override def doProduce(ctx: CodegenContext): String = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala index b94b0d26b2f26..89487c6b87150 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala @@ -71,8 +71,8 @@ case class BroadcastHashJoin( } } - override def upstreams(): Seq[RDD[InternalRow]] = { - streamedPlan.asInstanceOf[CodegenSupport].upstreams() + override def inputRDDs(): Seq[RDD[InternalRow]] = { + streamedPlan.asInstanceOf[CodegenSupport].inputRDDs() } override def doProduce(ctx: CodegenContext): String = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala index 443a7b43b6e9a..4e45fd656007f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala @@ -216,7 +216,7 @@ case class SortMergeJoin( joinType == Inner } - override def upstreams(): Seq[RDD[InternalRow]] = { + override def inputRDDs(): Seq[RDD[InternalRow]] = { left.execute() :: right.execute() :: Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index 9643b52f96544..c9a14593fb400 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -57,8 +57,8 @@ trait BaseLimit extends UnaryNode with CodegenSupport { iter.take(limit) } - override def upstreams(): Seq[RDD[InternalRow]] = { - child.asInstanceOf[CodegenSupport].upstreams() + override def inputRDDs(): Seq[RDD[InternalRow]] = { + child.asInstanceOf[CodegenSupport].inputRDDs() } protected override def doProduce(ctx: CodegenContext): String = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index 784b1e8c26fe1..e7261fc512ba2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -36,8 +36,8 @@ case class DeserializeToObject( child: SparkPlan) extends UnaryNode with CodegenSupport { override def output: Seq[Attribute] = deserializer.toAttribute :: Nil - override def upstreams(): Seq[RDD[InternalRow]] = { - child.asInstanceOf[CodegenSupport].upstreams() + override def inputRDDs(): Seq[RDD[InternalRow]] = { + child.asInstanceOf[CodegenSupport].inputRDDs() } protected override def doProduce(ctx: CodegenContext): String = { @@ -69,8 +69,8 @@ case class SerializeFromObject( child: SparkPlan) extends UnaryNode with CodegenSupport { override def output: Seq[Attribute] = serializer.map(_.toAttribute) - override def upstreams(): Seq[RDD[InternalRow]] = { - child.asInstanceOf[CodegenSupport].upstreams() + override def inputRDDs(): Seq[RDD[InternalRow]] = { + child.asInstanceOf[CodegenSupport].inputRDDs() } protected override def doProduce(ctx: CodegenContext): String = { @@ -153,8 +153,8 @@ case class MapElements( child: SparkPlan) extends UnaryNode with ObjectOperator with CodegenSupport { override def output: Seq[Attribute] = serializer.map(_.toAttribute) - override def upstreams(): Seq[RDD[InternalRow]] = { - child.asInstanceOf[CodegenSupport].upstreams() + override def inputRDDs(): Seq[RDD[InternalRow]] = { + child.asInstanceOf[CodegenSupport].inputRDDs() } protected override def doProduce(ctx: CodegenContext): String = { From d4b94ead92177a18d78a9701cfde9979641d2a18 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 18 Apr 2016 23:48:22 -0700 Subject: [PATCH 25/69] [SPARK-14595][SQL] add input metrics for FileScanRDD ## What changes were proposed in this pull request? This is roughly based on the input metrics logic in `SqlNewHadoopRDD` ## How was this patch tested? Not sure how to write a test, I manually verified it in Spark UI. Author: Wenchen Fan Closes #12352 from cloud-fan/metrics. --- .../execution/datasources/FileScanRDD.scala | 60 ++++++++++++++++--- 1 file changed, 53 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index 468e101fedb8b..f86911e002a44 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -18,14 +18,16 @@ package org.apache.spark.sql.execution.datasources import org.apache.spark.{Partition, TaskContext} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.{InputFileNameHolder, RDD} import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.vectorized.ColumnarBatch /** * A single file that should be read, along with partition column values that * need to be prepended to each row. The reading should start at the first - * valid record found after `offset`. + * valid record found after `start`. */ case class PartitionedFile( partitionValues: InternalRow, @@ -53,33 +55,77 @@ class FileScanRDD( override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { val iterator = new Iterator[Object] with AutoCloseable { + private val inputMetrics = context.taskMetrics().inputMetrics + private val existingBytesRead = inputMetrics.bytesRead + + // Find a function that will return the FileSystem bytes read by this thread. Do this before + // apply readFunction, because it might read some bytes. + private val getBytesReadCallback: Option[() => Long] = + SparkHadoopUtil.get.getFSBytesReadOnThreadCallback() + + // For Hadoop 2.5+, we get our input bytes from thread-local Hadoop FileSystem statistics. + // If we do a coalesce, however, we are likely to compute multiple partitions in the same + // task and in the same thread, in which case we need to avoid override values written by + // previous partitions (SPARK-13071). + private def updateBytesRead(): Unit = { + getBytesReadCallback.foreach { getBytesRead => + inputMetrics.setBytesRead(existingBytesRead + getBytesRead()) + } + } + + // If we can't get the bytes read from the FS stats, fall back to the file size, + // which may be inaccurate. + private def updateBytesReadWithFileSize(): Unit = { + if (getBytesReadCallback.isEmpty && currentFile != null) { + inputMetrics.incBytesRead(currentFile.length) + } + } + private[this] val files = split.asInstanceOf[FilePartition].files.toIterator + private[this] var currentFile: PartitionedFile = null private[this] var currentIterator: Iterator[Object] = null def hasNext = (currentIterator != null && currentIterator.hasNext) || nextIterator() - def next() = currentIterator.next() + def next() = { + val nextElement = currentIterator.next() + // TODO: we should have a better separation of row based and batch based scan, so that we + // don't need to run this `if` for every record. + if (nextElement.isInstanceOf[ColumnarBatch]) { + inputMetrics.incRecordsRead(nextElement.asInstanceOf[ColumnarBatch].numRows()) + } else { + inputMetrics.incRecordsRead(1) + } + if (inputMetrics.recordsRead % SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS == 0) { + updateBytesRead() + } + nextElement + } /** Advances to the next file. Returns true if a new non-empty iterator is available. */ private def nextIterator(): Boolean = { + updateBytesReadWithFileSize() if (files.hasNext) { - val nextFile = files.next() - logInfo(s"Reading File $nextFile") - InputFileNameHolder.setInputFileName(nextFile.filePath) - currentIterator = readFunction(nextFile) + currentFile = files.next() + logInfo(s"Reading File $currentFile") + InputFileNameHolder.setInputFileName(currentFile.filePath) + currentIterator = readFunction(currentFile) hasNext } else { + currentFile = null InputFileNameHolder.unsetInputFileName() false } } override def close() = { + updateBytesRead() + updateBytesReadWithFileSize() InputFileNameHolder.unsetInputFileName() } } // Register an on-task-completion callback to close the input stream. - context.addTaskCompletionListener(context => iterator.close()) + context.addTaskCompletionListener(_ => iterator.close()) iterator.asInstanceOf[Iterator[InternalRow]] // This is an erasure hack. } From 74fe235ab5ed169fb30d9d2c04077b90d1bf1b95 Mon Sep 17 00:00:00 2001 From: bomeng Date: Tue, 19 Apr 2016 09:09:58 +0200 Subject: [PATCH 26/69] [SPARK-14398][SQL] Audit non-reserved keyword list in ANTLR4 parser ## What changes were proposed in this pull request? I have compared non-reserved list in Antlr3 and Antlr4 one by one as well as all the existing keywords defined in Antlr4, added the missing keywords to the non-reserved keywords list. If we need to support more syntax, we can add more keywords by then. Any recommendation for the above is welcome. ## How was this patch tested? I manually checked the keywords one by one. Please let me know if there is a better way to test. Another thought: I suggest to put all the keywords definition and non-reserved list in order, that will be much easier to check in the future. Author: bomeng Closes #12191 from bomeng/SPARK-14398. --- .../main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 9cf2dd257e5c1..6f104a1489b15 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -652,6 +652,7 @@ nonReserved | STATISTICS | ANALYZE | PARTITIONED | EXTERNAL | DEFINED | RECORDWRITER | REVOKE | GRANT | LOCK | UNLOCK | MSCK | REPAIR | EXPORT | IMPORT | LOAD | VALUES | COMMENT | ROLE | ROLES | COMPACTIONS | PRINCIPALS | TRANSACTIONS | INDEX | INDEXES | LOCKS | OPTION + | ASC | DESC | LIMIT | RENAME | SETS ; SELECT: 'SELECT'; From 3d46d796a3a2b60b37dc318652eded5e992be1e5 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 19 Apr 2016 21:38:15 +0800 Subject: [PATCH 27/69] [SPARK-14577][SQL] Add spark.sql.codegen.maxCaseBranches config option ## What changes were proposed in this pull request? We currently disable codegen for `CaseWhen` if the number of branches is greater than 20 (in CaseWhen.MAX_NUM_CASES_FOR_CODEGEN). It would be better if this value is a non-public config defined in SQLConf. ## How was this patch tested? Pass the Jenkins tests (including a new testcase `Support spark.sql.codegen.maxCaseBranches option`) Author: Dongjoon Hyun Closes #12353 from dongjoon-hyun/SPARK-14577. --- .../spark/sql/catalyst/CatalystConf.scala | 4 +- .../expressions/conditionalExpressions.scala | 86 +++++++++------ .../sql/catalyst/optimizer/Optimizer.scala | 14 ++- .../optimizer/OptimizeCodegenSuite.scala | 102 ++++++++++++++++++ .../sql/execution/WholeStageCodegen.scala | 1 - .../apache/spark/sql/internal/SQLConf.scala | 8 ++ 6 files changed, 180 insertions(+), 35 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeCodegenSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala index abba8668216f4..0efe3c4d456ee 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala @@ -29,6 +29,7 @@ trait CatalystConf { def groupByOrdinal: Boolean def optimizerMaxIterations: Int + def maxCaseBranchesForCodegen: Int /** * Returns the [[Resolver]] for the current configuration, which can be used to determine if two @@ -45,6 +46,7 @@ case class SimpleCatalystConf( caseSensitiveAnalysis: Boolean, orderByOrdinal: Boolean = true, groupByOrdinal: Boolean = true, - optimizerMaxIterations: Int = 100) + optimizerMaxIterations: Int = 100, + maxCaseBranchesForCodegen: Int = 20) extends CatalystConf { } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index 336649c0fd763..e97e08947a500 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -81,18 +81,15 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi } /** - * Case statements of the form "CASE WHEN a THEN b [WHEN c THEN d]* [ELSE e] END". - * When a = true, returns b; when c = true, returns d; else returns e. + * Abstract parent class for common logic in CaseWhen and CaseWhenCodegen. * * @param branches seq of (branch condition, branch value) * @param elseValue optional value for the else branch */ -// scalastyle:off line.size.limit -@ExpressionDescription( - usage = "CASE WHEN a THEN b [WHEN c THEN d]* [ELSE e] END - When a = true, returns b; when c = true, return d; else return e.") -// scalastyle:on line.size.limit -case class CaseWhen(branches: Seq[(Expression, Expression)], elseValue: Option[Expression] = None) - extends Expression with CodegenFallback { +abstract class CaseWhenBase( + branches: Seq[(Expression, Expression)], + elseValue: Option[Expression]) + extends Expression with Serializable { override def children: Seq[Expression] = branches.flatMap(b => b._1 :: b._2 :: Nil) ++ elseValue @@ -142,16 +139,58 @@ case class CaseWhen(branches: Seq[(Expression, Expression)], elseValue: Option[E } } - def shouldCodegen: Boolean = { - branches.length < CaseWhen.MAX_NUM_CASES_FOR_CODEGEN + override def toString: String = { + val cases = branches.map { case (c, v) => s" WHEN $c THEN $v" }.mkString + val elseCase = elseValue.map(" ELSE " + _).getOrElse("") + "CASE" + cases + elseCase + " END" + } + + override def sql: String = { + val cases = branches.map { case (c, v) => s" WHEN ${c.sql} THEN ${v.sql}" }.mkString + val elseCase = elseValue.map(" ELSE " + _.sql).getOrElse("") + "CASE" + cases + elseCase + " END" + } +} + + +/** + * Case statements of the form "CASE WHEN a THEN b [WHEN c THEN d]* [ELSE e] END". + * When a = true, returns b; when c = true, returns d; else returns e. + * + * @param branches seq of (branch condition, branch value) + * @param elseValue optional value for the else branch + */ +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = "CASE WHEN a THEN b [WHEN c THEN d]* [ELSE e] END - When a = true, returns b; when c = true, return d; else return e.") +// scalastyle:on line.size.limit +case class CaseWhen( + val branches: Seq[(Expression, Expression)], + val elseValue: Option[Expression] = None) + extends CaseWhenBase(branches, elseValue) with CodegenFallback with Serializable { + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + super[CodegenFallback].doGenCode(ctx, ev) + } + + def toCodegen(): CaseWhenCodegen = { + CaseWhenCodegen(branches, elseValue) } +} + +/** + * CaseWhen expression used when code generation condition is satisfied. + * OptimizeCodegen optimizer replaces CaseWhen into CaseWhenCodegen. + * + * @param branches seq of (branch condition, branch value) + * @param elseValue optional value for the else branch + */ +case class CaseWhenCodegen( + val branches: Seq[(Expression, Expression)], + val elseValue: Option[Expression] = None) + extends CaseWhenBase(branches, elseValue) with Serializable { override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - if (!shouldCodegen) { - // Fallback to interpreted mode if there are too many branches, as it may reach the - // 64K limit (limit on bytecode size for a single function). - return super[CodegenFallback].doGenCode(ctx, ev) - } // Generate code that looks like: // // condA = ... @@ -202,26 +241,10 @@ case class CaseWhen(branches: Seq[(Expression, Expression)], elseValue: Option[E ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; $generatedCode""") } - - override def toString: String = { - val cases = branches.map { case (c, v) => s" WHEN $c THEN $v" }.mkString - val elseCase = elseValue.map(" ELSE " + _).getOrElse("") - "CASE" + cases + elseCase + " END" - } - - override def sql: String = { - val cases = branches.map { case (c, v) => s" WHEN ${c.sql} THEN ${v.sql}" }.mkString - val elseCase = elseValue.map(" ELSE " + _.sql).getOrElse("") - "CASE" + cases + elseCase + " END" - } } /** Factory methods for CaseWhen. */ object CaseWhen { - - // The maximum number of switches supported with codegen. - val MAX_NUM_CASES_FOR_CODEGEN = 20 - def apply(branches: Seq[(Expression, Expression)], elseValue: Expression): CaseWhen = { CaseWhen(branches, Option(elseValue)) } @@ -242,7 +265,6 @@ object CaseWhen { } } - /** * Case statements of the form "CASE a WHEN b THEN c [WHEN d THEN e]* [ELSE f] END". * When a = b, returns c; when a = d, returns e; else returns f. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index c46bdfb2b506a..b806b725a8d0c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -104,7 +104,9 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) Batch("LocalRelation", fixedPoint, ConvertToLocalRelation) :: Batch("Subquery", Once, - OptimizeSubqueries) :: Nil + OptimizeSubqueries) :: + Batch("OptimizeCodegen", Once, + OptimizeCodegen(conf)) :: Nil } /** @@ -863,6 +865,16 @@ object SimplifyConditionals extends Rule[LogicalPlan] with PredicateHelper { } } +/** + * Optimizes expressions by replacing according to CodeGen configuration. + */ +case class OptimizeCodegen(conf: CatalystConf) extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + case e @ CaseWhen(branches, _) if branches.size < conf.maxCaseBranchesForCodegen => + e.toCodegen() + } +} + /** * Combines all adjacent [[Union]] operators into a single [[Union]]. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeCodegenSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeCodegenSuite.scala new file mode 100644 index 0000000000000..4385b0e019f25 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeCodegenSuite.scala @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.SimpleCatalystConf +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.Literal._ +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules._ + + +class OptimizeCodegenSuite extends PlanTest { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = Batch("OptimizeCodegen", Once, OptimizeCodegen(SimpleCatalystConf(true))) :: Nil + } + + protected def assertEquivalent(e1: Expression, e2: Expression): Unit = { + val correctAnswer = Project(Alias(e2, "out")() :: Nil, OneRowRelation).analyze + val actual = Optimize.execute(Project(Alias(e1, "out")() :: Nil, OneRowRelation).analyze) + comparePlans(actual, correctAnswer) + } + + test("Codegen only when the number of branches is small.") { + assertEquivalent( + CaseWhen(Seq((TrueLiteral, Literal(1))), Literal(2)), + CaseWhen(Seq((TrueLiteral, Literal(1))), Literal(2)).toCodegen()) + + assertEquivalent( + CaseWhen(List.fill(100)(TrueLiteral, Literal(1)), Literal(2)), + CaseWhen(List.fill(100)(TrueLiteral, Literal(1)), Literal(2))) + } + + test("Nested CaseWhen Codegen.") { + assertEquivalent( + CaseWhen( + Seq((CaseWhen(Seq((TrueLiteral, Literal(1))), Literal(2)), Literal(3))), + CaseWhen(Seq((TrueLiteral, Literal(4))), Literal(5))), + CaseWhen( + Seq((CaseWhen(Seq((TrueLiteral, Literal(1))), Literal(2)).toCodegen(), Literal(3))), + CaseWhen(Seq((TrueLiteral, Literal(4))), Literal(5)).toCodegen()).toCodegen()) + } + + test("Multiple CaseWhen in one operator.") { + val plan = OneRowRelation + .select( + CaseWhen(Seq((TrueLiteral, Literal(1))), Literal(2)), + CaseWhen(Seq((FalseLiteral, Literal(3))), Literal(4)), + CaseWhen(List.fill(20)((TrueLiteral, Literal(0))), Literal(0)), + CaseWhen(Seq((TrueLiteral, Literal(5))), Literal(6))).analyze + val correctAnswer = OneRowRelation + .select( + CaseWhen(Seq((TrueLiteral, Literal(1))), Literal(2)).toCodegen(), + CaseWhen(Seq((FalseLiteral, Literal(3))), Literal(4)).toCodegen(), + CaseWhen(List.fill(20)((TrueLiteral, Literal(0))), Literal(0)), + CaseWhen(Seq((TrueLiteral, Literal(5))), Literal(6)).toCodegen()).analyze + val optimized = Optimize.execute(plan) + comparePlans(optimized, correctAnswer) + } + + test("Multiple CaseWhen in different operators") { + val plan = OneRowRelation + .select( + CaseWhen(Seq((TrueLiteral, Literal(1))), Literal(2)), + CaseWhen(Seq((FalseLiteral, Literal(3))), Literal(4)), + CaseWhen(List.fill(20)((TrueLiteral, Literal(0))), Literal(0))) + .where( + LessThan( + CaseWhen(Seq((TrueLiteral, Literal(5))), Literal(6)), + CaseWhen(List.fill(20)((TrueLiteral, Literal(0))), Literal(0))) + ).analyze + val correctAnswer = OneRowRelation + .select( + CaseWhen(Seq((TrueLiteral, Literal(1))), Literal(2)).toCodegen(), + CaseWhen(Seq((FalseLiteral, Literal(3))), Literal(4)).toCodegen(), + CaseWhen(List.fill(20)((TrueLiteral, Literal(0))), Literal(0))) + .where( + LessThan( + CaseWhen(Seq((TrueLiteral, Literal(5))), Literal(6)).toCodegen(), + CaseWhen(List.fill(20)((TrueLiteral, Literal(0))), Literal(0))) + ).analyze + val optimized = Optimize.execute(plan) + comparePlans(optimized, correctAnswer) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala index 29b66e3dee3e7..46eaede5e717b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala @@ -429,7 +429,6 @@ case class CollapseCodegenStages(conf: SQLConf) extends Rule[SparkPlan] { private def supportCodegen(e: Expression): Boolean = e match { case e: LeafExpression => true - case e: CaseWhen => e.shouldCodegen // CodegenFallback requires the input to be an InternalRow case e: CodegenFallback => false case _ => true diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 7f206bdb9b240..4ae8278a9d767 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -402,6 +402,12 @@ object SQLConf { .intConf .createWithDefault(200) + val MAX_CASES_BRANCHES = SQLConfigBuilder("spark.sql.codegen.maxCaseBranches") + .internal() + .doc("The maximum number of switches supported with codegen.") + .intConf + .createWithDefault(20) + val FILES_MAX_PARTITION_BYTES = SQLConfigBuilder("spark.sql.files.maxPartitionBytes") .doc("The maximum number of bytes to pack into a single partition when reading files.") .longConf @@ -529,6 +535,8 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def wholeStageMaxNumFields: Int = getConf(WHOLESTAGE_MAX_NUM_FIELDS) + def maxCaseBranchesForCodegen: Int = getConf(MAX_CASES_BRANCHES) + def exchangeReuseEnabled: Boolean = getConf(EXCHANGE_REUSE_ENABLED) def canonicalView: Boolean = getConf(CANONICAL_NATIVE_VIEW) From 5e360c93bed9d4f9250cf79bbcebd8552557f548 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 19 Apr 2016 09:37:00 -0700 Subject: [PATCH 28/69] [SPARK-13681][SPARK-14458][SPARK-14566][SQL] Add back once removed CommitFailureTestRelationSuite and SimpleTextHadoopFsRelationSuite ## What changes were proposed in this pull request? These test suites were removed while refactoring `HadoopFsRelation` related API. This PR brings them back. This PR also fixes two regressions: - SPARK-14458, which causes runtime error when saving partitioned tables using `FileFormat` data sources that are not able to infer their own schemata. This bug wasn't detected by any built-in data sources because all of them happen to have schema inference feature. - SPARK-14566, which happens to be covered by SPARK-14458 and causes wrong query result or runtime error when - appending a Dataset `ds` to a persisted partitioned data source relation `t`, and - partition columns in `ds` don't all appear after data columns ## How was this patch tested? `CommitFailureTestRelationSuite` uses a testing relation that always fails when committing write tasks to test write job cleanup. `SimpleTextHadoopFsRelationSuite` uses a testing relation to test general `HadoopFsRelation` and `FileFormat` interfaces. The two regressions are both covered by existing test cases. Author: Cheng Lian Closes #12179 from liancheng/spark-13681-commit-failure-test. --- .../execution/datasources/DataSource.scala | 14 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 11 +- .../spark/sql/hive/execution/commands.scala | 2 +- .../CommitFailureTestRelationSuite.scala | 82 +++++++++ .../sql/sources/CommitFailureTestSource.scala | 67 +++++++ .../SimpleTextHadoopFsRelationSuite.scala | 68 +++++++ .../sql/sources/SimpleTextRelation.scala | 167 ++++++++++++++++++ 7 files changed, 403 insertions(+), 8 deletions(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestRelationSuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextHadoopFsRelationSuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 10fde152ab2a9..23a7071086087 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -310,7 +310,17 @@ case class DataSource( val fileCatalog: FileCatalog = new HDFSFileCatalog(sqlContext, options, globbedPaths, partitionSchema) - val dataSchema = userSpecifiedSchema.orElse { + + val dataSchema = userSpecifiedSchema.map { schema => + val equality = + if (sqlContext.conf.caseSensitiveAnalysis) { + org.apache.spark.sql.catalyst.analysis.caseSensitiveResolution + } else { + org.apache.spark.sql.catalyst.analysis.caseInsensitiveResolution + } + + StructType(schema.filterNot(f => partitionColumns.exists(equality(_, f.name)))) + }.orElse { format.inferSchema( sqlContext, caseInsensitiveOptions, @@ -318,7 +328,7 @@ case class DataSource( }.getOrElse { throw new AnalysisException( s"Unable to infer schema for $format at ${allPaths.take(2).mkString(",")}. " + - "It must be specified manually") + "It must be specified manually") } val enrichedOptions = diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 697cf719c164e..79fe23b258c19 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -504,11 +504,12 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte } } - private def convertToLogicalRelation(metastoreRelation: MetastoreRelation, - options: Map[String, String], - defaultSource: FileFormat, - fileFormatClass: Class[_ <: FileFormat], - fileType: String): LogicalRelation = { + private def convertToLogicalRelation( + metastoreRelation: MetastoreRelation, + options: Map[String, String], + defaultSource: FileFormat, + fileFormatClass: Class[_ <: FileFormat], + fileType: String): LogicalRelation = { val metastoreSchema = StructType.fromAttributes(metastoreRelation.output) val tableIdentifier = QualifiedTableName(metastoreRelation.databaseName, metastoreRelation.tableName) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 5ef502afa5e67..8f7c4e8289900 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -300,7 +300,7 @@ case class CreateMetastoreDataSourceAsSelect( val data = Dataset.ofRows(hiveContext, query) val df = existingSchema match { // If we are inserting into an existing table, just use the existing schema. - case Some(s) => sqlContext.internalCreateDataFrame(data.queryExecution.toRdd, s) + case Some(s) => data.selectExpr(s.fieldNames: _*) case None => data } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestRelationSuite.scala new file mode 100644 index 0000000000000..08e83b7f6905a --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestRelationSuite.scala @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources + +import org.apache.hadoop.fs.Path + +import org.apache.spark.SparkException +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils + +class CommitFailureTestRelationSuite extends SQLTestUtils with TestHiveSingleton { + // When committing a task, `CommitFailureTestSource` throws an exception for testing purpose. + val dataSourceName: String = classOf[CommitFailureTestSource].getCanonicalName + + test("SPARK-7684: commitTask() failure should fallback to abortTask()") { + withTempPath { file => + // Here we coalesce partition number to 1 to ensure that only a single task is issued. This + // prevents race condition happened when FileOutputCommitter tries to remove the `_temporary` + // directory while committing/aborting the job. See SPARK-8513 for more details. + val df = sqlContext.range(0, 10).coalesce(1) + intercept[SparkException] { + df.write.format(dataSourceName).save(file.getCanonicalPath) + } + + val fs = new Path(file.getCanonicalPath).getFileSystem(SparkHadoopUtil.get.conf) + assert(!fs.exists(new Path(file.getCanonicalPath, "_temporary"))) + } + } + + test("call failure callbacks before close writer - default") { + SimpleTextRelation.failCommitter = false + withTempPath { file => + // fail the job in the middle of writing + val divideByZero = udf((x: Int) => { x / (x - 1)}) + val df = sqlContext.range(0, 10).coalesce(1).select(divideByZero(col("id"))) + + SimpleTextRelation.callbackCalled = false + intercept[SparkException] { + df.write.format(dataSourceName).save(file.getCanonicalPath) + } + assert(SimpleTextRelation.callbackCalled, "failure callback should be called") + + val fs = new Path(file.getCanonicalPath).getFileSystem(SparkHadoopUtil.get.conf) + assert(!fs.exists(new Path(file.getCanonicalPath, "_temporary"))) + } + } + + test("call failure callbacks before close writer - partitioned") { + SimpleTextRelation.failCommitter = false + withTempPath { file => + // fail the job in the middle of writing + val df = sqlContext.range(0, 10).coalesce(1).select(col("id").mod(2).as("key"), col("id")) + + SimpleTextRelation.callbackCalled = false + SimpleTextRelation.failWriter = true + intercept[SparkException] { + df.write.format(dataSourceName).partitionBy("key").save(file.getCanonicalPath) + } + assert(SimpleTextRelation.callbackCalled, "failure callback should be called") + + val fs = new Path(file.getCanonicalPath).getFileSystem(SparkHadoopUtil.get.conf) + assert(!fs.exists(new Path(file.getCanonicalPath, "_temporary"))) + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala new file mode 100644 index 0000000000000..6d7e7b77df81e --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources + +import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} + +import org.apache.spark.TaskContext +import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.types.StructType + +class CommitFailureTestSource extends SimpleTextSource { + /** + * Prepares a write job and returns an [[OutputWriterFactory]]. Client side job preparation can + * be put here. For example, user defined output committer can be configured here + * by setting the output committer class in the conf of spark.sql.sources.outputCommitterClass. + */ + override def prepareWrite( + sqlContext: SQLContext, + job: Job, + options: Map[String, String], + dataSchema: StructType): OutputWriterFactory = + new OutputWriterFactory { + override def newInstance( + path: String, + bucketId: Option[Int], + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = { + new SimpleTextOutputWriter(path, context) { + var failed = false + TaskContext.get().addTaskFailureListener { (t: TaskContext, e: Throwable) => + failed = true + SimpleTextRelation.callbackCalled = true + } + + override def write(row: Row): Unit = { + if (SimpleTextRelation.failWriter) { + sys.error("Intentional task writer failure for testing purpose.") + + } + super.write(row) + } + + override def close(): Unit = { + super.close() + sys.error("Intentional task commitment failure for testing purpose.") + } + } + } + } + + override def shortName(): String = "commit-failure-test" +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextHadoopFsRelationSuite.scala new file mode 100644 index 0000000000000..71e3457d2596a --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextHadoopFsRelationSuite.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources + +import org.apache.hadoop.fs.Path + +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.sql.catalyst.expressions.PredicateHelper +import org.apache.spark.sql.types._ + +class SimpleTextHadoopFsRelationSuite extends HadoopFsRelationTest with PredicateHelper { + override val dataSourceName: String = classOf[SimpleTextSource].getCanonicalName + + // We have a very limited number of supported types at here since it is just for a + // test relation and we do very basic testing at here. + override protected def supportsDataType(dataType: DataType): Boolean = dataType match { + case _: BinaryType => false + // We are using random data generator and the generated strings are not really valid string. + case _: StringType => false + case _: BooleanType => false // see https://issues.apache.org/jira/browse/SPARK-10442 + case _: CalendarIntervalType => false + case _: DateType => false + case _: TimestampType => false + case _: ArrayType => false + case _: MapType => false + case _: StructType => false + case _: UserDefinedType[_] => false + case _ => true + } + + test("save()/load() - partitioned table - simple queries - partition columns in data") { + withTempDir { file => + val basePath = new Path(file.getCanonicalPath) + val fs = basePath.getFileSystem(SparkHadoopUtil.get.conf) + val qualifiedBasePath = fs.makeQualified(basePath) + + for (p1 <- 1 to 2; p2 <- Seq("foo", "bar")) { + val partitionDir = new Path(qualifiedBasePath, s"p1=$p1/p2=$p2") + sparkContext + .parallelize(for (i <- 1 to 3) yield s"$i,val_$i,$p1") + .saveAsTextFile(partitionDir.toString) + } + + val dataSchemaWithPartition = + StructType(dataSchema.fields :+ StructField("p1", IntegerType, nullable = true)) + + checkQueries( + hiveContext.read.format(dataSourceName) + .option("dataSchema", dataSchemaWithPartition.json) + .load(file.getCanonicalPath)) + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala new file mode 100644 index 0000000000000..113b124be30d9 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala @@ -0,0 +1,167 @@ +/* + * 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.sources + +import java.text.NumberFormat + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.io.{NullWritable, Text} +import org.apache.hadoop.mapreduce.{Job, RecordWriter, TaskAttemptContext} +import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat, TextOutputFormat} + +import org.apache.spark.sql.{sources, Row, SQLContext} +import org.apache.spark.sql.catalyst.{expressions, InternalRow} +import org.apache.spark.sql.catalyst.expressions.{Cast, Expression, GenericInternalRow, InterpretedPredicate, InterpretedProjection, JoinedRow, Literal} +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.execution.datasources.{HadoopFileLinesReader, PartitionedFile} +import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.util.SerializableConfiguration + +class SimpleTextSource extends FileFormat with DataSourceRegister { + override def shortName(): String = "test" + + override def inferSchema( + sqlContext: SQLContext, + options: Map[String, String], + files: Seq[FileStatus]): Option[StructType] = { + Some(DataType.fromJson(options("dataSchema")).asInstanceOf[StructType]) + } + + override def prepareWrite( + sqlContext: SQLContext, + job: Job, + options: Map[String, String], + dataSchema: StructType): OutputWriterFactory = new OutputWriterFactory { + override private[sql] def newInstance( + path: String, + bucketId: Option[Int], + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = { + new SimpleTextOutputWriter(path, context) + } + } + + override def buildReader( + sqlContext: SQLContext, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String]): (PartitionedFile) => Iterator[InternalRow] = { + + SimpleTextRelation.requiredColumns = requiredSchema.fieldNames + SimpleTextRelation.pushedFilters = filters.toSet + + val fieldTypes = dataSchema.map(_.dataType) + val inputAttributes = dataSchema.toAttributes + val outputAttributes = requiredSchema.flatMap { field => + inputAttributes.find(_.name == field.name) + } + + val conf = new Configuration(sqlContext.sparkContext.hadoopConfiguration) + val broadcastedConf = + sqlContext.sparkContext.broadcast(new SerializableConfiguration(conf)) + + (file: PartitionedFile) => { + val predicate = { + val filterCondition: Expression = filters.collect { + // According to `unhandledFilters`, `SimpleTextRelation` only handles `GreaterThan` filter + case sources.GreaterThan(column, value) => + val dataType = dataSchema(column).dataType + val literal = Literal.create(value, dataType) + val attribute = inputAttributes.find(_.name == column).get + expressions.GreaterThan(attribute, literal) + }.reduceOption(expressions.And).getOrElse(Literal(true)) + InterpretedPredicate.create(filterCondition, inputAttributes) + } + + // Uses a simple projection to simulate column pruning + val projection = new InterpretedProjection(outputAttributes, inputAttributes) + + val unsafeRowIterator = + new HadoopFileLinesReader(file, broadcastedConf.value.value).map { line => + val record = line.toString + new GenericInternalRow(record.split(",", -1).zip(fieldTypes).map { + case (v, dataType) => + val value = if (v == "") null else v + // `Cast`ed values are always of internal types (e.g. UTF8String instead of String) + Cast(Literal(value), dataType).eval() + }) + }.filter(predicate).map(projection) + + // Appends partition values + val fullOutput = requiredSchema.toAttributes ++ partitionSchema.toAttributes + val joinedRow = new JoinedRow() + val appendPartitionColumns = GenerateUnsafeProjection.generate(fullOutput, fullOutput) + + unsafeRowIterator.map { dataRow => + appendPartitionColumns(joinedRow(dataRow, file.partitionValues)) + } + } + } +} + +class SimpleTextOutputWriter(path: String, context: TaskAttemptContext) extends OutputWriter { + private val recordWriter: RecordWriter[NullWritable, Text] = + new AppendingTextOutputFormat(new Path(path)).getRecordWriter(context) + + override def write(row: Row): Unit = { + val serialized = row.toSeq.map { v => + if (v == null) "" else v.toString + }.mkString(",") + recordWriter.write(null, new Text(serialized)) + } + + override def close(): Unit = { + recordWriter.close(context) + } +} + +class AppendingTextOutputFormat(outputFile: Path) extends TextOutputFormat[NullWritable, Text] { + val numberFormat = NumberFormat.getInstance() + + numberFormat.setMinimumIntegerDigits(5) + numberFormat.setGroupingUsed(false) + + override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { + val configuration = context.getConfiguration + val uniqueWriteJobId = configuration.get("spark.sql.sources.writeJobUUID") + val taskAttemptId = context.getTaskAttemptID + val split = taskAttemptId.getTaskID.getId + val name = FileOutputFormat.getOutputName(context) + new Path(outputFile, s"$name-${numberFormat.format(split)}-$uniqueWriteJobId") + } +} + +object SimpleTextRelation { + // Used to test column pruning + var requiredColumns: Seq[String] = Nil + + // Used to test filter push-down + var pushedFilters: Set[Filter] = Set.empty + + // Used to test failed committer + var failCommitter = false + + // Used to test failed writer + var failWriter = false + + // Used to test failure callback + var callbackCalled = false +} From 9ee95b6eccba41460b79c0aced9d00a39b5ae0c3 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 19 Apr 2016 10:00:44 -0700 Subject: [PATCH 29/69] [SPARK-14491] [SQL] refactor object operator framework to make it easy to eliminate serializations ## What changes were proposed in this pull request? This PR tries to separate the serialization and deserialization logic from object operators, so that it's easier to eliminate unnecessary serializations in optimizer. Typed aggregate related operators are special, they will deserialize the input row to multiple objects and it's difficult to simply use a deserializer operator to abstract it, so we still mix the deserialization logic there. ## How was this patch tested? existing tests and new test in `EliminateSerializationSuite` Author: Wenchen Fan Closes #12260 from cloud-fan/encoder. --- .../sql/catalyst/analysis/Analyzer.scala | 6 +- .../spark/sql/catalyst/dsl/package.scala | 4 + .../sql/catalyst/optimizer/Optimizer.scala | 31 +--- .../sql/catalyst/plans/logical/object.scala | 166 +++++++++--------- .../EliminateSerializationSuite.scala | 62 ++++--- .../scala/org/apache/spark/sql/Dataset.scala | 10 +- .../spark/sql/execution/SparkStrategies.scala | 22 +-- .../sql/execution/WholeStageCodegen.scala | 4 + .../apache/spark/sql/execution/objects.scala | 160 ++++++++++------- .../org/apache/spark/sql/QueryTest.scala | 4 +- .../execution/WholeStageCodegenSuite.scala | 2 +- 11 files changed, 254 insertions(+), 217 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 6591559426c68..0e2fd43983ef1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1672,9 +1672,9 @@ object CleanupAliases extends Rule[LogicalPlan] { // Operators that operate on objects should only have expressions from encoders, which should // never have extra aliases. - case o: ObjectOperator => o - case d: DeserializeToObject => d - case s: SerializeFromObject => s + case o: ObjectConsumer => o + case o: ObjectProducer => o + case a: AppendColumns => a case other => var stop = false diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 958966328bbe2..085e95f542a16 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -245,6 +245,10 @@ package object dsl { def struct(attrs: AttributeReference*): AttributeReference = struct(StructType.fromAttributes(attrs)) + /** Creates a new AttributeReference of object type */ + def obj(cls: Class[_]): AttributeReference = + AttributeReference(s, ObjectType(cls), nullable = true)() + /** Create a function. */ def function(exprs: Expression*): UnresolvedFunction = UnresolvedFunction(s, exprs, isDistinct = false) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index b806b725a8d0c..0a5232b2d485d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -153,29 +153,16 @@ object SamplePushDown extends Rule[LogicalPlan] { * representation of data item. For example back to back map operations. */ object EliminateSerialization extends Rule[LogicalPlan] { - // TODO: find a more general way to do this optimization. def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case m @ MapPartitions(_, deserializer, _, child: ObjectOperator) - if !deserializer.isInstanceOf[Attribute] && - deserializer.dataType == child.outputObject.dataType => - val childWithoutSerialization = child.withObjectOutput - m.copy( - deserializer = childWithoutSerialization.output.head, - child = childWithoutSerialization) - - case m @ MapElements(_, deserializer, _, child: ObjectOperator) - if !deserializer.isInstanceOf[Attribute] && - deserializer.dataType == child.outputObject.dataType => - val childWithoutSerialization = child.withObjectOutput - m.copy( - deserializer = childWithoutSerialization.output.head, - child = childWithoutSerialization) - - case d @ DeserializeToObject(_, s: SerializeFromObject) + case d @ DeserializeToObject(_, _, s: SerializeFromObject) if d.outputObjectType == s.inputObjectType => // Adds an extra Project here, to preserve the output expr id of `DeserializeToObject`. val objAttr = Alias(s.child.output.head, "obj")(exprId = d.output.head.exprId) Project(objAttr :: Nil, s.child) + + case a @ AppendColumns(_, _, _, s: SerializeFromObject) + if a.deserializer.dataType == s.inputObjectType => + AppendColumnsWithObject(a.func, s.serializer, a.serializer, s.child) } } @@ -366,9 +353,9 @@ object ColumnPruning extends Rule[LogicalPlan] { } a.copy(child = Expand(newProjects, newOutput, grandChild)) - // Prunes the unused columns from child of MapPartitions - case mp @ MapPartitions(_, _, _, child) if (child.outputSet -- mp.references).nonEmpty => - mp.copy(child = prunedChild(child, mp.references)) + // Prunes the unused columns from child of `DeserializeToObject` + case d @ DeserializeToObject(_, _, child) if (child.outputSet -- d.references).nonEmpty => + d.copy(child = prunedChild(child, d.references)) // Prunes the unused columns from child of Aggregate/Expand/Generate case a @ Aggregate(_, _, child) if (child.outputSet -- a.references).nonEmpty => @@ -1453,7 +1440,7 @@ object EmbedSerializerInFilter extends Rule[LogicalPlan] { s } else { val newCondition = condition transform { - case a: Attribute if a == d.output.head => d.deserializer.child + case a: Attribute if a == d.output.head => d.deserializer } Filter(newCondition, d.child) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala index 6df46189b627c..4a1bdb0b8ac2e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala @@ -21,126 +21,111 @@ import org.apache.spark.sql.Encoder import org.apache.spark.sql.catalyst.analysis.UnresolvedDeserializer import org.apache.spark.sql.catalyst.encoders._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types.{DataType, ObjectType, StructType} +import org.apache.spark.sql.types.{DataType, StructType} object CatalystSerde { def deserialize[T : Encoder](child: LogicalPlan): DeserializeToObject = { val deserializer = UnresolvedDeserializer(encoderFor[T].deserializer) - DeserializeToObject(Alias(deserializer, "obj")(), child) + DeserializeToObject(deserializer, generateObjAttr[T], child) } def serialize[T : Encoder](child: LogicalPlan): SerializeFromObject = { SerializeFromObject(encoderFor[T].namedExpressions, child) } + + def generateObjAttr[T : Encoder]: Attribute = { + AttributeReference("obj", encoderFor[T].deserializer.dataType, nullable = false)() + } } /** - * Takes the input row from child and turns it into object using the given deserializer expression. - * The output of this operator is a single-field safe row containing the deserialized object. + * A trait for logical operators that produces domain objects as output. + * The output of this operator is a single-field safe row containing the produced object. */ -case class DeserializeToObject( - deserializer: Alias, - child: LogicalPlan) extends UnaryNode { - override def output: Seq[Attribute] = deserializer.toAttribute :: Nil +trait ObjectProducer extends LogicalPlan { + // The attribute that reference to the single object field this operator outputs. + protected def outputObjAttr: Attribute + + override def output: Seq[Attribute] = outputObjAttr :: Nil + + override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) - def outputObjectType: DataType = deserializer.dataType + def outputObjectType: DataType = outputObjAttr.dataType } /** - * Takes the input object from child and turns in into unsafe row using the given serializer - * expression. The output of its child must be a single-field row containing the input object. + * A trait for logical operators that consumes domain objects as input. + * The output of its child must be a single-field row containing the input object. */ -case class SerializeFromObject( - serializer: Seq[NamedExpression], - child: LogicalPlan) extends UnaryNode { - override def output: Seq[Attribute] = serializer.map(_.toAttribute) +trait ObjectConsumer extends UnaryNode { + assert(child.output.length == 1) + + // This operator always need all columns of its child, even it doesn't reference to. + override def references: AttributeSet = child.outputSet def inputObjectType: DataType = child.output.head.dataType } /** - * A trait for logical operators that apply user defined functions to domain objects. + * Takes the input row from child and turns it into object using the given deserializer expression. */ -trait ObjectOperator extends LogicalPlan { +case class DeserializeToObject( + deserializer: Expression, + outputObjAttr: Attribute, + child: LogicalPlan) extends UnaryNode with ObjectProducer - /** The serializer that is used to produce the output of this operator. */ - def serializer: Seq[NamedExpression] +/** + * Takes the input object from child and turns it into unsafe row using the given serializer + * expression. + */ +case class SerializeFromObject( + serializer: Seq[NamedExpression], + child: LogicalPlan) extends UnaryNode with ObjectConsumer { override def output: Seq[Attribute] = serializer.map(_.toAttribute) - - /** - * The object type that is produced by the user defined function. Note that the return type here - * is the same whether or not the operator is output serialized data. - */ - def outputObject: NamedExpression = - Alias(serializer.head.collect { case b: BoundReference => b }.head, "obj")() - - /** - * Returns a copy of this operator that will produce an object instead of an encoded row. - * Used in the optimizer when transforming plans to remove unneeded serialization. - */ - def withObjectOutput: LogicalPlan = if (output.head.dataType.isInstanceOf[ObjectType]) { - this - } else { - withNewSerializer(outputObject :: Nil) - } - - /** Returns a copy of this operator with a different serializer. */ - def withNewSerializer(newSerializer: Seq[NamedExpression]): LogicalPlan = makeCopy { - productIterator.map { - case c if c == serializer => newSerializer - case other: AnyRef => other - }.toArray - } } object MapPartitions { def apply[T : Encoder, U : Encoder]( func: Iterator[T] => Iterator[U], - child: LogicalPlan): MapPartitions = { - MapPartitions( + child: LogicalPlan): LogicalPlan = { + val deserialized = CatalystSerde.deserialize[T](child) + val mapped = MapPartitions( func.asInstanceOf[Iterator[Any] => Iterator[Any]], - UnresolvedDeserializer(encoderFor[T].deserializer), - encoderFor[U].namedExpressions, - child) + CatalystSerde.generateObjAttr[U], + deserialized) + CatalystSerde.serialize[U](mapped) } } /** * A relation produced by applying `func` to each partition of the `child`. - * - * @param deserializer used to extract the input to `func` from an input row. - * @param serializer use to serialize the output of `func`. */ case class MapPartitions( func: Iterator[Any] => Iterator[Any], - deserializer: Expression, - serializer: Seq[NamedExpression], - child: LogicalPlan) extends UnaryNode with ObjectOperator + outputObjAttr: Attribute, + child: LogicalPlan) extends UnaryNode with ObjectConsumer with ObjectProducer object MapElements { def apply[T : Encoder, U : Encoder]( func: AnyRef, - child: LogicalPlan): MapElements = { - MapElements( + child: LogicalPlan): LogicalPlan = { + val deserialized = CatalystSerde.deserialize[T](child) + val mapped = MapElements( func, - UnresolvedDeserializer(encoderFor[T].deserializer), - encoderFor[U].namedExpressions, - child) + CatalystSerde.generateObjAttr[U], + deserialized) + CatalystSerde.serialize[U](mapped) } } /** * A relation produced by applying `func` to each element of the `child`. - * - * @param deserializer used to extract the input to `func` from an input row. - * @param serializer use to serialize the output of `func`. */ case class MapElements( func: AnyRef, - deserializer: Expression, - serializer: Seq[NamedExpression], - child: LogicalPlan) extends UnaryNode with ObjectOperator + outputObjAttr: Attribute, + child: LogicalPlan) extends UnaryNode with ObjectConsumer with ObjectProducer /** Factory for constructing new `AppendColumn` nodes. */ object AppendColumns { @@ -156,7 +141,7 @@ object AppendColumns { } /** - * A relation produced by applying `func` to each partition of the `child`, concatenating the + * A relation produced by applying `func` to each element of the `child`, concatenating the * resulting columns at the end of the input row. * * @param deserializer used to extract the input to `func` from an input row. @@ -166,28 +151,41 @@ case class AppendColumns( func: Any => Any, deserializer: Expression, serializer: Seq[NamedExpression], - child: LogicalPlan) extends UnaryNode with ObjectOperator { + child: LogicalPlan) extends UnaryNode { override def output: Seq[Attribute] = child.output ++ newColumns def newColumns: Seq[Attribute] = serializer.map(_.toAttribute) } +/** + * An optimized version of [[AppendColumns]], that can be executed on deserialized object directly. + */ +case class AppendColumnsWithObject( + func: Any => Any, + childSerializer: Seq[NamedExpression], + newColumnsSerializer: Seq[NamedExpression], + child: LogicalPlan) extends UnaryNode with ObjectConsumer { + + override def output: Seq[Attribute] = (childSerializer ++ newColumnsSerializer).map(_.toAttribute) +} + /** Factory for constructing new `MapGroups` nodes. */ object MapGroups { def apply[K : Encoder, T : Encoder, U : Encoder]( func: (K, Iterator[T]) => TraversableOnce[U], groupingAttributes: Seq[Attribute], dataAttributes: Seq[Attribute], - child: LogicalPlan): MapGroups = { - new MapGroups( + child: LogicalPlan): LogicalPlan = { + val mapped = new MapGroups( func.asInstanceOf[(Any, Iterator[Any]) => TraversableOnce[Any]], UnresolvedDeserializer(encoderFor[K].deserializer, groupingAttributes), UnresolvedDeserializer(encoderFor[T].deserializer, dataAttributes), - encoderFor[U].namedExpressions, groupingAttributes, dataAttributes, + CatalystSerde.generateObjAttr[U], child) + CatalystSerde.serialize[U](mapped) } } @@ -198,43 +196,43 @@ object MapGroups { * * @param keyDeserializer used to extract the key object for each group. * @param valueDeserializer used to extract the items in the iterator from an input row. - * @param serializer use to serialize the output of `func`. */ case class MapGroups( func: (Any, Iterator[Any]) => TraversableOnce[Any], keyDeserializer: Expression, valueDeserializer: Expression, - serializer: Seq[NamedExpression], groupingAttributes: Seq[Attribute], dataAttributes: Seq[Attribute], - child: LogicalPlan) extends UnaryNode with ObjectOperator + outputObjAttr: Attribute, + child: LogicalPlan) extends UnaryNode with ObjectProducer /** Factory for constructing new `CoGroup` nodes. */ object CoGroup { - def apply[Key : Encoder, Left : Encoder, Right : Encoder, Result : Encoder]( - func: (Key, Iterator[Left], Iterator[Right]) => TraversableOnce[Result], + def apply[K : Encoder, L : Encoder, R : Encoder, OUT : Encoder]( + func: (K, Iterator[L], Iterator[R]) => TraversableOnce[OUT], leftGroup: Seq[Attribute], rightGroup: Seq[Attribute], leftAttr: Seq[Attribute], rightAttr: Seq[Attribute], left: LogicalPlan, - right: LogicalPlan): CoGroup = { + right: LogicalPlan): LogicalPlan = { require(StructType.fromAttributes(leftGroup) == StructType.fromAttributes(rightGroup)) - CoGroup( + val cogrouped = CoGroup( func.asInstanceOf[(Any, Iterator[Any], Iterator[Any]) => TraversableOnce[Any]], // The `leftGroup` and `rightGroup` are guaranteed te be of same schema, so it's safe to // resolve the `keyDeserializer` based on either of them, here we pick the left one. - UnresolvedDeserializer(encoderFor[Key].deserializer, leftGroup), - UnresolvedDeserializer(encoderFor[Left].deserializer, leftAttr), - UnresolvedDeserializer(encoderFor[Right].deserializer, rightAttr), - encoderFor[Result].namedExpressions, + UnresolvedDeserializer(encoderFor[K].deserializer, leftGroup), + UnresolvedDeserializer(encoderFor[L].deserializer, leftAttr), + UnresolvedDeserializer(encoderFor[R].deserializer, rightAttr), leftGroup, rightGroup, leftAttr, rightAttr, + CatalystSerde.generateObjAttr[OUT], left, right) + CatalystSerde.serialize[OUT](cogrouped) } } @@ -247,10 +245,10 @@ case class CoGroup( keyDeserializer: Expression, leftDeserializer: Expression, rightDeserializer: Expression, - serializer: Seq[NamedExpression], leftGroup: Seq[Attribute], rightGroup: Seq[Attribute], leftAttr: Seq[Attribute], rightAttr: Seq[Attribute], + outputObjAttr: Attribute, left: LogicalPlan, - right: LogicalPlan) extends BinaryNode with ObjectOperator + right: LogicalPlan) extends BinaryNode with ObjectProducer diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSerializationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSerializationSuite.scala index 91777375608fd..3c033ddc374cf 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSerializationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSerializationSuite.scala @@ -22,8 +22,7 @@ import scala.reflect.runtime.universe.TypeTag import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.catalyst.expressions.NewInstance -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, MapPartitions} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.rules.RuleExecutor @@ -37,40 +36,45 @@ class EliminateSerializationSuite extends PlanTest { } implicit private def productEncoder[T <: Product : TypeTag] = ExpressionEncoder[T]() - private val func = identity[Iterator[(Int, Int)]] _ - private val func2 = identity[Iterator[OtherTuple]] _ + implicit private def intEncoder = ExpressionEncoder[Int]() - def assertObjectCreations(count: Int, plan: LogicalPlan): Unit = { - val newInstances = plan.flatMap(_.expressions.collect { - case n: NewInstance => n - }) + test("back to back serialization") { + val input = LocalRelation('obj.obj(classOf[(Int, Int)])) + val plan = input.serialize[(Int, Int)].deserialize[(Int, Int)].analyze + val optimized = Optimize.execute(plan) + val expected = input.select('obj.as("obj")).analyze + comparePlans(optimized, expected) + } - if (newInstances.size != count) { - fail( - s""" - |Wrong number of object creations in plan: ${newInstances.size} != $count - |$plan - """.stripMargin) - } + test("back to back serialization with object change") { + val input = LocalRelation('obj.obj(classOf[OtherTuple])) + val plan = input.serialize[OtherTuple].deserialize[(Int, Int)].analyze + val optimized = Optimize.execute(plan) + comparePlans(optimized, plan) } - test("back to back MapPartitions") { - val input = LocalRelation('_1.int, '_2.int) - val plan = - MapPartitions(func, - MapPartitions(func, input)) + test("back to back serialization in AppendColumns") { + val input = LocalRelation('obj.obj(classOf[(Int, Int)])) + val func = (item: (Int, Int)) => item._1 + val plan = AppendColumns(func, input.serialize[(Int, Int)]).analyze + + val optimized = Optimize.execute(plan) + + val expected = AppendColumnsWithObject( + func.asInstanceOf[Any => Any], + productEncoder[(Int, Int)].namedExpressions, + intEncoder.namedExpressions, + input).analyze - val optimized = Optimize.execute(plan.analyze) - assertObjectCreations(1, optimized) + comparePlans(optimized, expected) } - test("back to back with object change") { - val input = LocalRelation('_1.int, '_2.int) - val plan = - MapPartitions(func, - MapPartitions(func2, input)) + test("back to back serialization in AppendColumns with object change") { + val input = LocalRelation('obj.obj(classOf[OtherTuple])) + val func = (item: (Int, Int)) => item._1 + val plan = AppendColumns(func, input.serialize[OtherTuple]).analyze - val optimized = Optimize.execute(plan.analyze) - assertObjectCreations(2, optimized) + val optimized = Optimize.execute(plan) + comparePlans(optimized, plan) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 1a09d70fb94c3..3c708cbf29851 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2251,16 +2251,16 @@ class Dataset[T] private[sql]( def unpersist(): this.type = unpersist(blocking = false) /** - * Represents the content of the [[Dataset]] as an [[RDD]] of [[Row]]s. Note that the RDD is - * memoized. Once called, it won't change even if you change any query planning related Spark SQL - * configurations (e.g. `spark.sql.shuffle.partitions`). + * Represents the content of the [[Dataset]] as an [[RDD]] of [[T]]. * * @group rdd * @since 1.6.0 */ lazy val rdd: RDD[T] = { - queryExecution.toRdd.mapPartitions { rows => - rows.map(boundTEncoder.fromRow) + val objectType = unresolvedTEncoder.deserializer.dataType + val deserialized = CatalystSerde.deserialize[T](logicalPlan) + sqlContext.executePlan(deserialized).toRdd.mapPartitions { rows => + rows.map(_.get(0, objectType).asInstanceOf[T]) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index c15aaed3654ff..a4b0fa59dbb24 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -346,21 +346,23 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { throw new IllegalStateException( "logical intersect operator should have been replaced by semi-join in the optimizer") - case logical.DeserializeToObject(deserializer, child) => - execution.DeserializeToObject(deserializer, planLater(child)) :: Nil + case logical.DeserializeToObject(deserializer, objAttr, child) => + execution.DeserializeToObject(deserializer, objAttr, planLater(child)) :: Nil case logical.SerializeFromObject(serializer, child) => execution.SerializeFromObject(serializer, planLater(child)) :: Nil - case logical.MapPartitions(f, in, out, child) => - execution.MapPartitions(f, in, out, planLater(child)) :: Nil - case logical.MapElements(f, in, out, child) => - execution.MapElements(f, in, out, planLater(child)) :: Nil + case logical.MapPartitions(f, objAttr, child) => + execution.MapPartitions(f, objAttr, planLater(child)) :: Nil + case logical.MapElements(f, objAttr, child) => + execution.MapElements(f, objAttr, planLater(child)) :: Nil case logical.AppendColumns(f, in, out, child) => execution.AppendColumns(f, in, out, planLater(child)) :: Nil - case logical.MapGroups(f, key, in, out, grouping, data, child) => - execution.MapGroups(f, key, in, out, grouping, data, planLater(child)) :: Nil - case logical.CoGroup(f, keyObj, lObj, rObj, out, lGroup, rGroup, lAttr, rAttr, left, right) => + case logical.AppendColumnsWithObject(f, childSer, newSer, child) => + execution.AppendColumnsWithObject(f, childSer, newSer, planLater(child)) :: Nil + case logical.MapGroups(f, key, value, grouping, data, objAttr, child) => + execution.MapGroups(f, key, value, grouping, data, objAttr, planLater(child)) :: Nil + case logical.CoGroup(f, key, lObj, rObj, lGroup, rGroup, lAttr, rAttr, oAttr, left, right) => execution.CoGroup( - f, keyObj, lObj, rObj, out, lGroup, rGroup, lAttr, rAttr, + f, key, lObj, rObj, lGroup, rGroup, lAttr, rAttr, oAttr, planLater(left), planLater(right)) :: Nil case logical.Repartition(numPartitions, shuffle, child) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala index 46eaede5e717b..23b2eabd0c809 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala @@ -473,6 +473,10 @@ case class CollapseCodegenStages(conf: SQLConf) extends Rule[SparkPlan] { * Inserts a WholeStageCodegen on top of those that support codegen. */ private def insertWholeStageCodegen(plan: SparkPlan): SparkPlan = plan match { + // For operators that will output domain object, do not insert WholeStageCodegen for it as + // domain object can not be written into unsafe row. + case plan if plan.output.length == 1 && plan.output.head.dataType.isInstanceOf[ObjectType] => + plan.withNewChildren(plan.children.map(insertWholeStageCodegen)) case plan: CodegenSupport if supportCodegen(plan) => WholeStageCodegen(insertInputAdapter(plan)) case other => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index e7261fc512ba2..7c8bc7fed8313 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -25,16 +25,19 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.types.ObjectType +import org.apache.spark.sql.types.{DataType, ObjectType} /** * Takes the input row from child and turns it into object using the given deserializer expression. * The output of this operator is a single-field safe row containing the deserialized object. */ case class DeserializeToObject( - deserializer: Alias, + deserializer: Expression, + outputObjAttr: Attribute, child: SparkPlan) extends UnaryNode with CodegenSupport { - override def output: Seq[Attribute] = deserializer.toAttribute :: Nil + + override def output: Seq[Attribute] = outputObjAttr :: Nil + override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) override def inputRDDs(): Seq[RDD[InternalRow]] = { child.asInstanceOf[CodegenSupport].inputRDDs() @@ -67,6 +70,7 @@ case class DeserializeToObject( case class SerializeFromObject( serializer: Seq[NamedExpression], child: SparkPlan) extends UnaryNode with CodegenSupport { + override def output: Seq[Attribute] = serializer.map(_.toAttribute) override def inputRDDs(): Seq[RDD[InternalRow]] = { @@ -98,60 +102,71 @@ case class SerializeFromObject( * Helper functions for physical operators that work with user defined objects. */ trait ObjectOperator extends SparkPlan { - def generateToObject(objExpr: Expression, inputSchema: Seq[Attribute]): InternalRow => Any = { - val objectProjection = GenerateSafeProjection.generate(objExpr :: Nil, inputSchema) - (i: InternalRow) => objectProjection(i).get(0, objExpr.dataType) + def deserializeRowToObject( + deserializer: Expression, + inputSchema: Seq[Attribute]): InternalRow => Any = { + val proj = GenerateSafeProjection.generate(deserializer :: Nil, inputSchema) + (i: InternalRow) => proj(i).get(0, deserializer.dataType) } - def generateToRow(serializer: Seq[Expression]): Any => InternalRow = { - val outputProjection = if (serializer.head.dataType.isInstanceOf[ObjectType]) { - GenerateSafeProjection.generate(serializer) - } else { - GenerateUnsafeProjection.generate(serializer) + def serializeObjectToRow(serializer: Seq[Expression]): Any => UnsafeRow = { + val proj = GenerateUnsafeProjection.generate(serializer) + val objType = serializer.head.collect { case b: BoundReference => b.dataType }.head + val objRow = new SpecificMutableRow(objType :: Nil) + (o: Any) => { + objRow(0) = o + proj(objRow) } - val inputType = serializer.head.collect { case b: BoundReference => b.dataType }.head - val outputRow = new SpecificMutableRow(inputType :: Nil) + } + + def wrapObjectToRow(objType: DataType): Any => InternalRow = { + val outputRow = new SpecificMutableRow(objType :: Nil) (o: Any) => { outputRow(0) = o - outputProjection(outputRow) + outputRow } } + + def unwrapObjectFromRow(objType: DataType): InternalRow => Any = { + (i: InternalRow) => i.get(0, objType) + } } /** - * Applies the given function to each input row and encodes the result. + * Applies the given function to input object iterator. + * The output of its child must be a single-field row containing the input object. */ case class MapPartitions( func: Iterator[Any] => Iterator[Any], - deserializer: Expression, - serializer: Seq[NamedExpression], + outputObjAttr: Attribute, child: SparkPlan) extends UnaryNode with ObjectOperator { - override def output: Seq[Attribute] = serializer.map(_.toAttribute) + + override def output: Seq[Attribute] = outputObjAttr :: Nil + override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) override protected def doExecute(): RDD[InternalRow] = { child.execute().mapPartitionsInternal { iter => - val getObject = generateToObject(deserializer, child.output) - val outputObject = generateToRow(serializer) + val getObject = unwrapObjectFromRow(child.output.head.dataType) + val outputObject = wrapObjectToRow(outputObjAttr.dataType) func(iter.map(getObject)).map(outputObject) } } } /** - * Applies the given function to each input row and encodes the result. + * Applies the given function to each input object. + * The output of its child must be a single-field row containing the input object. * - * Note that, each serializer expression needs the result object which is returned by the given - * function, as input. This operator uses some tricks to make sure we only calculate the result - * object once. We don't use [[Project]] directly as subexpression elimination doesn't work with - * whole stage codegen and it's confusing to show the un-common-subexpression-eliminated version of - * a project while explain. + * This operator is kind of a safe version of [[Project]], as it's output is custom object, we need + * to use safe row to contain it. */ case class MapElements( func: AnyRef, - deserializer: Expression, - serializer: Seq[NamedExpression], + outputObjAttr: Attribute, child: SparkPlan) extends UnaryNode with ObjectOperator with CodegenSupport { - override def output: Seq[Attribute] = serializer.map(_.toAttribute) + + override def output: Seq[Attribute] = outputObjAttr :: Nil + override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) override def inputRDDs(): Seq[RDD[InternalRow]] = { child.asInstanceOf[CodegenSupport].inputRDDs() @@ -167,23 +182,14 @@ case class MapElements( case _ => classOf[Any => Any] -> "apply" } val funcObj = Literal.create(func, ObjectType(funcClass)) - val resultObjType = serializer.head.collect { case b: BoundReference => b }.head.dataType - val callFunc = Invoke(funcObj, methodName, resultObjType, Seq(deserializer)) + val callFunc = Invoke(funcObj, methodName, outputObjAttr.dataType, child.output) val bound = ExpressionCanonicalizer.execute( BindReferences.bindReference(callFunc, child.output)) ctx.currentVars = input - val evaluated = bound.genCode(ctx) - - val resultObj = LambdaVariable(evaluated.value, evaluated.isNull, resultObjType) - val outputFields = serializer.map(_ transform { - case _: BoundReference => resultObj - }) - val resultVars = outputFields.map(_.genCode(ctx)) - s""" - ${evaluated.code} - ${consume(ctx, resultVars)} - """ + val resultVars = bound.genCode(ctx) :: Nil + + consume(ctx, resultVars) } override protected def doExecute(): RDD[InternalRow] = { @@ -191,9 +197,10 @@ case class MapElements( case m: MapFunction[_, _] => i => m.asInstanceOf[MapFunction[Any, Any]].call(i) case _ => func.asInstanceOf[Any => Any] } + child.execute().mapPartitionsInternal { iter => - val getObject = generateToObject(deserializer, child.output) - val outputObject = generateToRow(serializer) + val getObject = unwrapObjectFromRow(child.output.head.dataType) + val outputObject = wrapObjectToRow(outputObjAttr.dataType) iter.map(row => outputObject(callFunc(getObject(row)))) } } @@ -216,15 +223,43 @@ case class AppendColumns( override protected def doExecute(): RDD[InternalRow] = { child.execute().mapPartitionsInternal { iter => - val getObject = generateToObject(deserializer, child.output) + val getObject = deserializeRowToObject(deserializer, child.output) val combiner = GenerateUnsafeRowJoiner.create(child.schema, newColumnSchema) - val outputObject = generateToRow(serializer) + val outputObject = serializeObjectToRow(serializer) iter.map { row => val newColumns = outputObject(func(getObject(row))) + combiner.join(row.asInstanceOf[UnsafeRow], newColumns): InternalRow + } + } + } +} + +/** + * An optimized version of [[AppendColumns]], that can be executed on deserialized object directly. + */ +case class AppendColumnsWithObject( + func: Any => Any, + inputSerializer: Seq[NamedExpression], + newColumnsSerializer: Seq[NamedExpression], + child: SparkPlan) extends UnaryNode with ObjectOperator { + + override def output: Seq[Attribute] = (inputSerializer ++ newColumnsSerializer).map(_.toAttribute) - // This operates on the assumption that we always serialize the result... - combiner.join(row.asInstanceOf[UnsafeRow], newColumns.asInstanceOf[UnsafeRow]): InternalRow + private def inputSchema = inputSerializer.map(_.toAttribute).toStructType + private def newColumnSchema = newColumnsSerializer.map(_.toAttribute).toStructType + + override protected def doExecute(): RDD[InternalRow] = { + child.execute().mapPartitionsInternal { iter => + val getChildObject = unwrapObjectFromRow(child.output.head.dataType) + val outputChildObject = serializeObjectToRow(inputSerializer) + val outputNewColumnOjb = serializeObjectToRow(newColumnsSerializer) + val combiner = GenerateUnsafeRowJoiner.create(inputSchema, newColumnSchema) + + iter.map { row => + val childObj = getChildObject(row) + val newColumns = outputNewColumnOjb(func(childObj)) + combiner.join(outputChildObject(childObj), newColumns): InternalRow } } } @@ -232,19 +267,19 @@ case class AppendColumns( /** * Groups the input rows together and calls the function with each group and an iterator containing - * all elements in the group. The result of this function is encoded and flattened before - * being output. + * all elements in the group. The result of this function is flattened before being output. */ case class MapGroups( func: (Any, Iterator[Any]) => TraversableOnce[Any], keyDeserializer: Expression, valueDeserializer: Expression, - serializer: Seq[NamedExpression], groupingAttributes: Seq[Attribute], dataAttributes: Seq[Attribute], + outputObjAttr: Attribute, child: SparkPlan) extends UnaryNode with ObjectOperator { - override def output: Seq[Attribute] = serializer.map(_.toAttribute) + override def output: Seq[Attribute] = outputObjAttr :: Nil + override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) override def requiredChildDistribution: Seq[Distribution] = ClusteredDistribution(groupingAttributes) :: Nil @@ -256,9 +291,9 @@ case class MapGroups( child.execute().mapPartitionsInternal { iter => val grouped = GroupedIterator(iter, groupingAttributes, child.output) - val getKey = generateToObject(keyDeserializer, groupingAttributes) - val getValue = generateToObject(valueDeserializer, dataAttributes) - val outputObject = generateToRow(serializer) + val getKey = deserializeRowToObject(keyDeserializer, groupingAttributes) + val getValue = deserializeRowToObject(valueDeserializer, dataAttributes) + val outputObject = wrapObjectToRow(outputObjAttr.dataType) grouped.flatMap { case (key, rowIter) => val result = func( @@ -273,22 +308,23 @@ case class MapGroups( /** * Co-groups the data from left and right children, and calls the function with each group and 2 * iterators containing all elements in the group from left and right side. - * The result of this function is encoded and flattened before being output. + * The result of this function is flattened before being output. */ case class CoGroup( func: (Any, Iterator[Any], Iterator[Any]) => TraversableOnce[Any], keyDeserializer: Expression, leftDeserializer: Expression, rightDeserializer: Expression, - serializer: Seq[NamedExpression], leftGroup: Seq[Attribute], rightGroup: Seq[Attribute], leftAttr: Seq[Attribute], rightAttr: Seq[Attribute], + outputObjAttr: Attribute, left: SparkPlan, right: SparkPlan) extends BinaryNode with ObjectOperator { - override def output: Seq[Attribute] = serializer.map(_.toAttribute) + override def output: Seq[Attribute] = outputObjAttr :: Nil + override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) override def requiredChildDistribution: Seq[Distribution] = ClusteredDistribution(leftGroup) :: ClusteredDistribution(rightGroup) :: Nil @@ -301,10 +337,10 @@ case class CoGroup( val leftGrouped = GroupedIterator(leftData, leftGroup, left.output) val rightGrouped = GroupedIterator(rightData, rightGroup, right.output) - val getKey = generateToObject(keyDeserializer, leftGroup) - val getLeft = generateToObject(leftDeserializer, leftAttr) - val getRight = generateToObject(rightDeserializer, rightAttr) - val outputObject = generateToRow(serializer) + val getKey = deserializeRowToObject(keyDeserializer, leftGroup) + val getLeft = deserializeRowToObject(leftDeserializer, leftAttr) + val getRight = deserializeRowToObject(rightDeserializer, rightAttr) + val outputObject = wrapObjectToRow(outputObjAttr.dataType) new CoGroupedIterator(leftGrouped, rightGrouped, leftGroup).flatMap { case (key, leftResult, rightResult) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 23a0ce215ff3f..2dca792c83bf0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -201,7 +201,9 @@ abstract class QueryTest extends PlanTest { val logicalPlan = df.queryExecution.analyzed // bypass some cases that we can't handle currently. logicalPlan.transform { - case _: ObjectOperator => return + case _: ObjectConsumer => return + case _: ObjectProducer => return + case _: AppendColumns => return case _: LogicalRelation => return case _: MemoryPlan => return }.transformAllExpressions { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index 8efd9de29eb0f..d7cf1dc6aadb4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -79,7 +79,7 @@ class WholeStageCodegenSuite extends SparkPlanTest with SharedSQLContext { val plan = ds.queryExecution.executedPlan assert(plan.find(p => p.isInstanceOf[WholeStageCodegen] && - p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[MapElements]).isDefined) + p.asInstanceOf[WholeStageCodegen].child.isInstanceOf[SerializeFromObject]).isDefined) assert(ds.collect() === 0.until(10).map(_.toString).toArray) } From e89633605ea14547dcece35a7e12a9f0f78c5181 Mon Sep 17 00:00:00 2001 From: tedyu Date: Tue, 19 Apr 2016 10:12:36 -0700 Subject: [PATCH 30/69] [SPARK-13904] Add exit code parameter to exitExecutor() ## What changes were proposed in this pull request? This PR adds exit code parameter to exitExecutor() so that caller can specify different exit code. ## How was this patch tested? Existing test rxin hbhanawat Author: tedyu Closes #12457 from tedyu/master. --- .../executor/CoarseGrainedExecutorBackend.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index db5b774806f94..e08729510926b 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -64,7 +64,7 @@ private[spark] class CoarseGrainedExecutorBackend( // Always receive `true`. Just ignore it case Failure(e) => logError(s"Cannot register with driver: $driverUrl", e) - exitExecutor() + exitExecutor(1) }(ThreadUtils.sameThread) } @@ -81,12 +81,12 @@ private[spark] class CoarseGrainedExecutorBackend( case RegisterExecutorFailed(message) => logError("Slave registration failed: " + message) - exitExecutor() + exitExecutor(1) case LaunchTask(data) => if (executor == null) { logError("Received LaunchTask command but executor was null") - exitExecutor() + exitExecutor(1) } else { val taskDesc = ser.deserialize[TaskDescription](data.value) logInfo("Got assigned task " + taskDesc.taskId) @@ -97,7 +97,7 @@ private[spark] class CoarseGrainedExecutorBackend( case KillTask(taskId, _, interruptThread) => if (executor == null) { logError("Received KillTask command but executor was null") - exitExecutor() + exitExecutor(1) } else { executor.killTask(taskId, interruptThread) } @@ -127,7 +127,7 @@ private[spark] class CoarseGrainedExecutorBackend( logInfo(s"Driver from $remoteAddress disconnected during shutdown") } else if (driver.exists(_.address == remoteAddress)) { logError(s"Driver $remoteAddress disassociated! Shutting down.") - exitExecutor() + exitExecutor(1) } else { logWarning(s"An unknown ($remoteAddress) driver disconnected.") } @@ -146,7 +146,7 @@ private[spark] class CoarseGrainedExecutorBackend( * executor exits differently. For e.g. when an executor goes down, * back-end may not want to take the parent process down. */ - protected def exitExecutor(): Unit = System.exit(1) + protected def exitExecutor(code: Int): Unit = System.exit(code) } private[spark] object CoarseGrainedExecutorBackend extends Logging { From d9620e769e41541347db863907bdbd057db50823 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Tue, 19 Apr 2016 10:33:40 -0700 Subject: [PATCH 31/69] [SPARK-12457] Fixed the Wrong Description and Missing Example in Collection Functions #### What changes were proposed in this pull request? https://github.com/apache/spark/pull/12185 contains the original PR I submitted in https://github.com/apache/spark/pull/10418 However, it misses one of the extended example, a wrong description and a few typos for collection functions. This PR is fix all these issues. #### How was this patch tested? The existing test cases already cover it. Author: gatorsmile Closes #12492 from gatorsmile/expressionUpdate. --- .../sql/catalyst/expressions/collectionOperations.scala | 9 +++++---- .../spark/sql/catalyst/plans/logical/commands.scala | 4 ++-- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index 864288394e66b..c71cb73d65bf6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -27,7 +27,8 @@ import org.apache.spark.sql.types._ * Given an array or map, returns its size. */ @ExpressionDescription( - usage = "_FUNC_(expr) - Returns the size of an array or a map.") + usage = "_FUNC_(expr) - Returns the size of an array or a map.", + extended = " > SELECT _FUNC_(array('b', 'd', 'c', 'a'));\n 4") case class Size(child: Expression) extends UnaryExpression with ExpectsInputTypes { override def dataType: DataType = IntegerType override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(ArrayType, MapType)) @@ -48,8 +49,8 @@ case class Size(child: Expression) extends UnaryExpression with ExpectsInputType */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(array(obj1, obj2,...)) - Sorts the input array in ascending order according to the natural ordering of the array elements.", - extended = " > SELECT _FUNC_(array('b', 'd', 'c', 'a'));\n 'a', 'b', 'c', 'd'") + usage = "_FUNC_(array(obj1, obj2, ...), ascendingOrder) - Sorts the input array in ascending order according to the natural ordering of the array elements.", + extended = " > SELECT _FUNC_(array('b', 'd', 'c', 'a'), true);\n 'a', 'b', 'c', 'd'") // scalastyle:on line.size.limit case class SortArray(base: Expression, ascendingOrder: Expression) extends BinaryExpression with ExpectsInputTypes with CodegenFallback { @@ -133,7 +134,7 @@ case class SortArray(base: Expression, ascendingOrder: Expression) * Checks if the array (left) has the element (right) */ @ExpressionDescription( - usage = "_FUNC_(array, value) - Returns TRUE if the array contains value.", + usage = "_FUNC_(array, value) - Returns TRUE if the array contains the value.", extended = " > SELECT _FUNC_(array(1, 2, 3), 2);\n true") case class ArrayContains(left: Expression, right: Expression) extends BinaryExpression with ImplicitCastInputTypes { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala index 47b34d1fa2e49..fcffdbaaf07b7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala @@ -28,9 +28,9 @@ import org.apache.spark.sql.types.StringType trait Command /** - * Returned for the "DESCRIBE [EXTENDED] FUNCTION functionName" command. + * Returned for the "DESCRIBE FUNCTION [EXTENDED] functionName" command. * @param functionName The function to be described. - * @param isExtended True if "DESCRIBE EXTENDED" is used. Otherwise, false. + * @param isExtended True if "DESCRIBE FUNCTION EXTENDED" is used. Otherwise, false. */ private[sql] case class DescribeFunction( functionName: String, From 947b9020b0d621bc97661a0a056297e6889936d3 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 19 Apr 2016 10:38:10 -0700 Subject: [PATCH 32/69] [SPARK-14676] Wrap and re-throw Await.result exceptions in order to capture full stacktrace When `Await.result` throws an exception which originated from a different thread, the resulting stacktrace doesn't include the path leading to the `Await.result` call itself, making it difficult to identify the impact of these exceptions. For example, I've seen cases where broadcast cleaning errors propagate to the main thread and crash it but the resulting stacktrace doesn't include any of the main thread's code, making it difficult to pinpoint which exception crashed that thread. This patch addresses this issue by explicitly catching, wrapping, and re-throwing exceptions that are thrown by `Await.result`. I tested this manually using https://github.com/JoshRosen/spark/commit/16b31c825197ee31a50214c6ba3c1df08148f403, a patch which reproduces an issue where an RPC exception which occurs while unpersisting RDDs manages to crash the main thread without any useful stacktrace, and verified that informative, full stacktraces were generated after applying the fix in this PR. /cc rxin nongli yhuai anabranch Author: Josh Rosen Closes #12433 from JoshRosen/wrap-and-rethrow-await-exceptions. --- .../scala/org/apache/spark/FutureAction.scala | 7 ++- .../spark/deploy/FaultToleranceTest.scala | 10 ++--- .../apache/spark/deploy/master/Master.scala | 4 +- .../deploy/rest/RestSubmissionClient.scala | 7 ++- .../spark/network/BlockTransferService.scala | 8 ++-- .../org/apache/spark/rpc/RpcTimeout.scala | 25 ++++++++--- .../apache/spark/storage/BlockManager.scala | 14 +++++- .../org/apache/spark/util/ThreadUtils.scala | 22 +++++++++- .../scala/org/apache/spark/util/Utils.scala | 2 + .../org/apache/spark/FutureActionSuite.scala | 7 +-- .../apache/spark/HeartbeatReceiverSuite.scala | 7 ++- .../apache/spark/JobCancellationSuite.scala | 14 +++--- .../spark/memory/MemoryManagerSuite.scala | 43 ++++++++++--------- .../spark/rdd/AsyncRDDActionsSuite.scala | 12 +++--- .../org/apache/spark/rpc/RpcEnvSuite.scala | 25 ++++++----- .../spark/rpc/netty/NettyRpcEnvSuite.scala | 7 +-- .../OutputCommitCoordinatorSuite.scala | 8 ++-- .../spark/storage/BlockInfoManagerSuite.scala | 22 +++++----- .../apache/spark/util/ThreadUtilsSuite.scala | 4 +- scalastyle-config.xml | 11 +++++ .../expressions/CodeGenerationSuite.scala | 3 +- .../spark/sql/execution/SparkPlan.scala | 3 +- .../exchange/BroadcastExchange.scala | 5 +-- .../sql/hive/thriftserver/CliSuite.scala | 6 +-- .../HiveThriftServer2Suites.scala | 15 ++++--- .../receiver/ReceivedBlockHandler.scala | 4 +- .../streaming/util/BatchedWriteAheadLog.scala | 7 +-- .../streaming/util/WriteAheadLogSuite.scala | 5 ++- 28 files changed, 191 insertions(+), 116 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index 339266a5d48b2..a50600f1488c9 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -28,6 +28,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.JavaFutureAction import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.JobWaiter +import org.apache.spark.util.ThreadUtils /** @@ -45,6 +46,7 @@ trait FutureAction[T] extends Future[T] { /** * Blocks until this action completes. + * * @param atMost maximum wait time, which may be negative (no waiting is done), Duration.Inf * for unbounded waiting, or a finite positive duration * @return this FutureAction @@ -53,6 +55,7 @@ trait FutureAction[T] extends Future[T] { /** * Awaits and returns the result (of type T) of this action. + * * @param atMost maximum wait time, which may be negative (no waiting is done), Duration.Inf * for unbounded waiting, or a finite positive duration * @throws Exception exception during action execution @@ -89,8 +92,8 @@ trait FutureAction[T] extends Future[T] { /** * Blocks and returns the result of this job. */ - @throws(classOf[Exception]) - def get(): T = Await.result(this, Duration.Inf) + @throws(classOf[SparkException]) + def get(): T = ThreadUtils.awaitResult(this, Duration.Inf) /** * Returns the job IDs run by the underlying async operation. diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala index abb98f95a1ee8..79f4d06c8460e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -23,7 +23,7 @@ import java.nio.charset.StandardCharsets import java.util.concurrent.TimeoutException import scala.collection.mutable.ListBuffer -import scala.concurrent.{Await, Future, Promise} +import scala.concurrent.{Future, Promise} import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ import scala.language.postfixOps @@ -35,7 +35,7 @@ import org.json4s.jackson.JsonMethods import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.master.RecoveryState import org.apache.spark.internal.Logging -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} /** * This suite tests the fault tolerance of the Spark standalone scheduler, mainly the Master. @@ -265,7 +265,7 @@ private object FaultToleranceTest extends App with Logging { } // Avoid waiting indefinitely (e.g., we could register but get no executors). - assertTrue(Await.result(f, 120 seconds)) + assertTrue(ThreadUtils.awaitResult(f, 120 seconds)) } /** @@ -318,7 +318,7 @@ private object FaultToleranceTest extends App with Logging { } try { - assertTrue(Await.result(f, 120 seconds)) + assertTrue(ThreadUtils.awaitResult(f, 120 seconds)) } catch { case e: TimeoutException => logError("Master states: " + masters.map(_.state)) @@ -422,7 +422,7 @@ private object SparkDocker { } dockerCmd.run(ProcessLogger(findIpAndLog _)) - val ip = Await.result(ipPromise.future, 30 seconds) + val ip = ThreadUtils.awaitResult(ipPromise.future, 30 seconds) val dockerId = Docker.getLastProcessId (ip, dockerId, outFile) } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index b443e8f0519f4..edc9be2a8a8cb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -24,7 +24,7 @@ import java.util.Date import java.util.concurrent.{ConcurrentHashMap, ScheduledFuture, TimeUnit} import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import scala.concurrent.{Await, ExecutionContext, Future} +import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration.Duration import scala.language.postfixOps import scala.util.Random @@ -959,7 +959,7 @@ private[deploy] class Master( */ private[master] def rebuildSparkUI(app: ApplicationInfo): Option[SparkUI] = { val futureUI = asyncRebuildSparkUI(app) - Await.result(futureUI, Duration.Inf) + ThreadUtils.awaitResult(futureUI, Duration.Inf) } /** Rebuild a new SparkUI asynchronously to not block RPC event loop */ diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala index c5a5876a896cc..21cb94142b15b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala @@ -27,10 +27,11 @@ import scala.collection.mutable import scala.concurrent.{Await, Future} import scala.concurrent.duration._ import scala.io.Source +import scala.util.control.NonFatal import com.fasterxml.jackson.core.JsonProcessingException -import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} +import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf, SparkException} import org.apache.spark.internal.Logging import org.apache.spark.util.Utils @@ -258,13 +259,17 @@ private[spark] class RestSubmissionClient(master: String) extends Logging { } } + // scalastyle:off awaitresult try { Await.result(responseFuture, 10.seconds) } catch { + // scalastyle:on awaitresult case unreachable @ (_: FileNotFoundException | _: SocketException) => throw new SubmitRestConnectionException("Unable to connect to server", unreachable) case malformed @ (_: JsonProcessingException | _: SubmitRestProtocolException) => throw new SubmitRestProtocolException("Malformed response received from server", malformed) case timeout: TimeoutException => throw new SubmitRestConnectionException("No response from server", timeout) + case NonFatal(t) => + throw new SparkException("Exception while waiting for response", t) } } diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala index 09ce012e4e692..cb9d389dd7ea6 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala @@ -20,7 +20,7 @@ package org.apache.spark.network import java.io.Closeable import java.nio.ByteBuffer -import scala.concurrent.{Await, Future, Promise} +import scala.concurrent.{Future, Promise} import scala.concurrent.duration.Duration import scala.reflect.ClassTag @@ -28,6 +28,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.shuffle.{BlockFetchingListener, ShuffleClient} import org.apache.spark.storage.{BlockId, StorageLevel} +import org.apache.spark.util.ThreadUtils private[spark] abstract class BlockTransferService extends ShuffleClient with Closeable with Logging { @@ -100,8 +101,7 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo result.success(new NioManagedBuffer(ret)) } }) - - Await.result(result.future, Duration.Inf) + ThreadUtils.awaitResult(result.future, Duration.Inf) } /** @@ -119,6 +119,6 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo level: StorageLevel, classTag: ClassTag[_]): Unit = { val future = uploadBlock(hostname, port, execId, blockId, blockData, level, classTag) - Await.result(future, Duration.Inf) + ThreadUtils.awaitResult(future, Duration.Inf) } } diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala b/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala index 2950df62bf285..2761d39e37029 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala @@ -19,10 +19,11 @@ package org.apache.spark.rpc import java.util.concurrent.TimeoutException -import scala.concurrent.{Await, Awaitable} +import scala.concurrent.{Await, Future} import scala.concurrent.duration._ +import scala.util.control.NonFatal -import org.apache.spark.SparkConf +import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.util.Utils /** @@ -65,14 +66,21 @@ private[spark] class RpcTimeout(val duration: FiniteDuration, val timeoutProp: S /** * Wait for the completed result and return it. If the result is not available within this * timeout, throw a [[RpcTimeoutException]] to indicate which configuration controls the timeout. - * @param awaitable the `Awaitable` to be awaited - * @throws RpcTimeoutException if after waiting for the specified time `awaitable` + * + * @param future the `Future` to be awaited + * @throws RpcTimeoutException if after waiting for the specified time `future` * is still not ready */ - def awaitResult[T](awaitable: Awaitable[T]): T = { + def awaitResult[T](future: Future[T]): T = { + val wrapAndRethrow: PartialFunction[Throwable, T] = { + case NonFatal(t) => + throw new SparkException("Exception thrown in awaitResult", t) + } try { - Await.result(awaitable, duration) - } catch addMessageIfTimeout + // scalastyle:off awaitresult + Await.result(future, duration) + // scalastyle:on awaitresult + } catch addMessageIfTimeout.orElse(wrapAndRethrow) } } @@ -82,6 +90,7 @@ private[spark] object RpcTimeout { /** * Lookup the timeout property in the configuration and create * a RpcTimeout with the property key in the description. + * * @param conf configuration properties containing the timeout * @param timeoutProp property key for the timeout in seconds * @throws NoSuchElementException if property is not set @@ -95,6 +104,7 @@ private[spark] object RpcTimeout { * Lookup the timeout property in the configuration and create * a RpcTimeout with the property key in the description. * Uses the given default value if property is not set + * * @param conf configuration properties containing the timeout * @param timeoutProp property key for the timeout in seconds * @param defaultValue default timeout value in seconds if property not found @@ -109,6 +119,7 @@ private[spark] object RpcTimeout { * and create a RpcTimeout with the first set property key in the * description. * Uses the given default value if property is not set + * * @param conf configuration properties containing the timeout * @param timeoutPropList prioritized list of property keys for the timeout in seconds * @param defaultValue default timeout value in seconds if no properties found diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 35a6c63ad193e..22bc76b143516 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -260,7 +260,12 @@ private[spark] class BlockManager( def waitForAsyncReregister(): Unit = { val task = asyncReregisterTask if (task != null) { - Await.ready(task, Duration.Inf) + try { + Await.ready(task, Duration.Inf) + } catch { + case NonFatal(t) => + throw new Exception("Error occurred while waiting for async. reregistration", t) + } } } @@ -802,7 +807,12 @@ private[spark] class BlockManager( logDebug("Put block %s locally took %s".format(blockId, Utils.getUsedTimeMs(startTimeMs))) if (level.replication > 1) { // Wait for asynchronous replication to finish - Await.ready(replicationFuture, Duration.Inf) + try { + Await.ready(replicationFuture, Duration.Inf) + } catch { + case NonFatal(t) => + throw new Exception("Error occurred while waiting for replication to finish", t) + } } if (blockWasSuccessfullyStored) { None diff --git a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala index 9abbf4a7a3971..5a6dbc830448a 100644 --- a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala @@ -19,12 +19,15 @@ package org.apache.spark.util import java.util.concurrent._ -import scala.concurrent.{ExecutionContext, ExecutionContextExecutor} +import scala.concurrent.{Await, Awaitable, ExecutionContext, ExecutionContextExecutor} +import scala.concurrent.duration.Duration import scala.concurrent.forkjoin.{ForkJoinPool => SForkJoinPool, ForkJoinWorkerThread => SForkJoinWorkerThread} import scala.util.control.NonFatal import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} +import org.apache.spark.SparkException + private[spark] object ThreadUtils { private val sameThreadExecutionContext = @@ -174,4 +177,21 @@ private[spark] object ThreadUtils { false // asyncMode ) } + + // scalastyle:off awaitresult + /** + * Preferred alternative to [[Await.result()]]. This method wraps and re-throws any exceptions + * thrown by the underlying [[Await]] call, ensuring that this thread's stack trace appears in + * logs. + */ + @throws(classOf[SparkException]) + def awaitResult[T](awaitable: Awaitable[T], atMost: Duration): T = { + try { + Await.result(awaitable, atMost) + // scalastyle:on awaitresult + } catch { + case NonFatal(t) => + throw new SparkException("Exception thrown in awaitResult: ", t) + } + } } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 78e164cff7738..848f7d7adbc7e 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1598,6 +1598,7 @@ private[spark] object Utils extends Logging { /** * Timing method based on iterations that permit JVM JIT optimization. + * * @param numIters number of iterations * @param f function to be executed. If prepare is not None, the running time of each call to f * must be an order of magnitude longer than one millisecond for accurate timing. @@ -1639,6 +1640,7 @@ private[spark] object Utils extends Logging { /** * Creates a symlink. + * * @param src absolute path to the source * @param dst relative path for the destination */ diff --git a/core/src/test/scala/org/apache/spark/FutureActionSuite.scala b/core/src/test/scala/org/apache/spark/FutureActionSuite.scala index 1102aea96b548..70b6309be7d53 100644 --- a/core/src/test/scala/org/apache/spark/FutureActionSuite.scala +++ b/core/src/test/scala/org/apache/spark/FutureActionSuite.scala @@ -17,11 +17,12 @@ package org.apache.spark -import scala.concurrent.Await import scala.concurrent.duration.Duration import org.scalatest.{BeforeAndAfter, Matchers} +import org.apache.spark.util.ThreadUtils + class FutureActionSuite extends SparkFunSuite @@ -36,7 +37,7 @@ class FutureActionSuite test("simple async action") { val rdd = sc.parallelize(1 to 10, 2) val job = rdd.countAsync() - val res = Await.result(job, Duration.Inf) + val res = ThreadUtils.awaitResult(job, Duration.Inf) res should be (10) job.jobIds.size should be (1) } @@ -44,7 +45,7 @@ class FutureActionSuite test("complex async action") { val rdd = sc.parallelize(1 to 15, 3) val job = rdd.takeAsync(10) - val res = Await.result(job, Duration.Inf) + val res = ThreadUtils.awaitResult(job, Duration.Inf) res should be (1 to 10) job.jobIds.size should be (2) } diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index 713d5e58b4ffc..4d2b3e7f3b14b 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -21,7 +21,6 @@ import java.util.concurrent.{ExecutorService, TimeUnit} import scala.collection.Map import scala.collection.mutable -import scala.concurrent.Await import scala.concurrent.duration._ import scala.language.postfixOps @@ -36,7 +35,7 @@ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.ManualClock +import org.apache.spark.util.{ManualClock, ThreadUtils} /** * A test suite for the heartbeating behavior between the driver and the executors. @@ -231,14 +230,14 @@ class HeartbeatReceiverSuite private def addExecutorAndVerify(executorId: String): Unit = { assert( heartbeatReceiver.addExecutor(executorId).map { f => - Await.result(f, 10.seconds) + ThreadUtils.awaitResult(f, 10.seconds) } === Some(true)) } private def removeExecutorAndVerify(executorId: String): Unit = { assert( heartbeatReceiver.removeExecutor(executorId).map { f => - Await.result(f, 10.seconds) + ThreadUtils.awaitResult(f, 10.seconds) } === Some(true)) } diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index c347ab8dc8020..a3490fc79e458 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark import java.util.concurrent.Semaphore -import scala.concurrent.Await import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ import scala.concurrent.Future @@ -28,6 +27,7 @@ import org.scalatest.BeforeAndAfter import org.scalatest.Matchers import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskStart} +import org.apache.spark.util.ThreadUtils /** * Test suite for cancelling running jobs. We run the cancellation tasks for single job action @@ -137,7 +137,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft sc.clearJobGroup() val jobB = sc.parallelize(1 to 100, 2).countAsync() sc.cancelJobGroup("jobA") - val e = intercept[SparkException] { Await.result(jobA, Duration.Inf) } + val e = intercept[SparkException] { ThreadUtils.awaitResult(jobA, Duration.Inf) }.getCause assert(e.getMessage contains "cancel") // Once A is cancelled, job B should finish fairly quickly. @@ -202,7 +202,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft sc.clearJobGroup() val jobB = sc.parallelize(1 to 100, 2).countAsync() sc.cancelJobGroup("jobA") - val e = intercept[SparkException] { Await.result(jobA, 5.seconds) } + val e = intercept[SparkException] { ThreadUtils.awaitResult(jobA, 5.seconds) }.getCause assert(e.getMessage contains "cancel") // Once A is cancelled, job B should finish fairly quickly. @@ -248,7 +248,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft { val f = sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.countAsync() Future { f.cancel() } - val e = intercept[SparkException] { f.get() } + val e = intercept[SparkException] { f.get() }.getCause assert(e.getMessage.contains("cancelled") || e.getMessage.contains("killed")) } @@ -268,7 +268,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft sem.acquire() f.cancel() } - val e = intercept[SparkException] { f.get() } + val e = intercept[SparkException] { f.get() }.getCause assert(e.getMessage.contains("cancelled") || e.getMessage.contains("killed")) } } @@ -278,7 +278,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft { val f = sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.takeAsync(5000) Future { f.cancel() } - val e = intercept[SparkException] { f.get() } + val e = intercept[SparkException] { f.get() }.getCause assert(e.getMessage.contains("cancelled") || e.getMessage.contains("killed")) } @@ -296,7 +296,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft sem.acquire() f.cancel() } - val e = intercept[SparkException] { f.get() } + val e = intercept[SparkException] { f.get() }.getCause assert(e.getMessage.contains("cancelled") || e.getMessage.contains("killed")) } } diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala index 99d5b496bcd2e..a1286523a235d 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.memory import java.util.concurrent.atomic.AtomicLong import scala.collection.mutable -import scala.concurrent.{Await, ExecutionContext, Future} +import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration.Duration import org.mockito.Matchers.{any, anyLong} @@ -33,6 +33,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkFunSuite import org.apache.spark.storage.{BlockId, BlockStatus, StorageLevel} import org.apache.spark.storage.memory.MemoryStore +import org.apache.spark.util.ThreadUtils /** @@ -172,15 +173,15 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft // Have both tasks request 500 bytes, then wait until both requests have been granted: val t1Result1 = Future { t1MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } - assert(Await.result(t1Result1, futureTimeout) === 500L) - assert(Await.result(t2Result1, futureTimeout) === 500L) + assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 500L) + assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 500L) // Have both tasks each request 500 bytes more; both should immediately return 0 as they are // both now at 1 / N val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } - assert(Await.result(t1Result2, 200.millis) === 0L) - assert(Await.result(t2Result2, 200.millis) === 0L) + assert(ThreadUtils.awaitResult(t1Result2, 200.millis) === 0L) + assert(ThreadUtils.awaitResult(t2Result2, 200.millis) === 0L) } test("two tasks cannot grow past 1 / N of on-heap execution memory") { @@ -192,15 +193,15 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft // Have both tasks request 250 bytes, then wait until both requests have been granted: val t1Result1 = Future { t1MemManager.acquireExecutionMemory(250L, MemoryMode.ON_HEAP, null) } val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L, MemoryMode.ON_HEAP, null) } - assert(Await.result(t1Result1, futureTimeout) === 250L) - assert(Await.result(t2Result1, futureTimeout) === 250L) + assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 250L) + assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 250L) // Have both tasks each request 500 bytes more. // We should only grant 250 bytes to each of them on this second request val t1Result2 = Future { t1MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } - assert(Await.result(t1Result2, futureTimeout) === 250L) - assert(Await.result(t2Result2, futureTimeout) === 250L) + assert(ThreadUtils.awaitResult(t1Result2, futureTimeout) === 250L) + assert(ThreadUtils.awaitResult(t2Result2, futureTimeout) === 250L) } test("tasks can block to get at least 1 / 2N of on-heap execution memory") { @@ -211,17 +212,17 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft // t1 grabs 1000 bytes and then waits until t2 is ready to make a request. val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L, MemoryMode.ON_HEAP, null) } - assert(Await.result(t1Result1, futureTimeout) === 1000L) + assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 1000L) val t2Result1 = Future { t2MemManager.acquireExecutionMemory(250L, MemoryMode.ON_HEAP, null) } // Make sure that t2 didn't grab the memory right away. This is hacky but it would be difficult // to make sure the other thread blocks for some time otherwise. Thread.sleep(300) t1MemManager.releaseExecutionMemory(250L, MemoryMode.ON_HEAP, null) // The memory freed from t1 should now be granted to t2. - assert(Await.result(t2Result1, futureTimeout) === 250L) + assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 250L) // Further requests by t2 should be denied immediately because it now has 1 / 2N of the memory. val t2Result2 = Future { t2MemManager.acquireExecutionMemory(100L, MemoryMode.ON_HEAP, null) } - assert(Await.result(t2Result2, 200.millis) === 0L) + assert(ThreadUtils.awaitResult(t2Result2, 200.millis) === 0L) } test("TaskMemoryManager.cleanUpAllAllocatedMemory") { @@ -232,18 +233,18 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft // t1 grabs 1000 bytes and then waits until t2 is ready to make a request. val t1Result1 = Future { t1MemManager.acquireExecutionMemory(1000L, MemoryMode.ON_HEAP, null) } - assert(Await.result(t1Result1, futureTimeout) === 1000L) + assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 1000L) val t2Result1 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } // Make sure that t2 didn't grab the memory right away. This is hacky but it would be difficult // to make sure the other thread blocks for some time otherwise. Thread.sleep(300) // t1 releases all of its memory, so t2 should be able to grab all of the memory t1MemManager.cleanUpAllAllocatedMemory() - assert(Await.result(t2Result1, futureTimeout) === 500L) + assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 500L) val t2Result2 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } - assert(Await.result(t2Result2, futureTimeout) === 500L) + assert(ThreadUtils.awaitResult(t2Result2, futureTimeout) === 500L) val t2Result3 = Future { t2MemManager.acquireExecutionMemory(500L, MemoryMode.ON_HEAP, null) } - assert(Await.result(t2Result3, 200.millis) === 0L) + assert(ThreadUtils.awaitResult(t2Result3, 200.millis) === 0L) } test("tasks should not be granted a negative amount of execution memory") { @@ -254,13 +255,13 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft val futureTimeout: Duration = 20.seconds val t1Result1 = Future { t1MemManager.acquireExecutionMemory(700L, MemoryMode.ON_HEAP, null) } - assert(Await.result(t1Result1, futureTimeout) === 700L) + assert(ThreadUtils.awaitResult(t1Result1, futureTimeout) === 700L) val t2Result1 = Future { t2MemManager.acquireExecutionMemory(300L, MemoryMode.ON_HEAP, null) } - assert(Await.result(t2Result1, futureTimeout) === 300L) + assert(ThreadUtils.awaitResult(t2Result1, futureTimeout) === 300L) val t1Result2 = Future { t1MemManager.acquireExecutionMemory(300L, MemoryMode.ON_HEAP, null) } - assert(Await.result(t1Result2, 200.millis) === 0L) + assert(ThreadUtils.awaitResult(t1Result2, 200.millis) === 0L) } test("off-heap execution allocations cannot exceed limit") { @@ -270,11 +271,11 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft val tMemManager = new TaskMemoryManager(memoryManager, 1) val result1 = Future { tMemManager.acquireExecutionMemory(1000L, MemoryMode.OFF_HEAP, null) } - assert(Await.result(result1, 200.millis) === 1000L) + assert(ThreadUtils.awaitResult(result1, 200.millis) === 1000L) assert(tMemManager.getMemoryConsumptionForThisTask === 1000L) val result2 = Future { tMemManager.acquireExecutionMemory(300L, MemoryMode.OFF_HEAP, null) } - assert(Await.result(result2, 200.millis) === 0L) + assert(ThreadUtils.awaitResult(result2, 200.millis) === 0L) assert(tMemManager.getMemoryConsumptionForThisTask === 1000L) tMemManager.releaseExecutionMemory(500L, MemoryMode.OFF_HEAP, null) diff --git a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala index d18bde790b40a..8cb0a295b0773 100644 --- a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala @@ -28,6 +28,7 @@ import org.scalatest.concurrent.Timeouts import org.scalatest.time.SpanSugar._ import org.apache.spark._ +import org.apache.spark.util.ThreadUtils class AsyncRDDActionsSuite extends SparkFunSuite with BeforeAndAfterAll with Timeouts { @@ -185,22 +186,23 @@ class AsyncRDDActionsSuite extends SparkFunSuite with BeforeAndAfterAll with Tim test("FutureAction result, infinite wait") { val f = sc.parallelize(1 to 100, 4) .countAsync() - assert(Await.result(f, Duration.Inf) === 100) + assert(ThreadUtils.awaitResult(f, Duration.Inf) === 100) } test("FutureAction result, finite wait") { val f = sc.parallelize(1 to 100, 4) .countAsync() - assert(Await.result(f, Duration(30, "seconds")) === 100) + assert(ThreadUtils.awaitResult(f, Duration(30, "seconds")) === 100) } test("FutureAction result, timeout") { val f = sc.parallelize(1 to 100, 4) .mapPartitions(itr => { Thread.sleep(20); itr }) .countAsync() - intercept[TimeoutException] { - Await.result(f, Duration(20, "milliseconds")) + val e = intercept[SparkException] { + ThreadUtils.awaitResult(f, Duration(20, "milliseconds")) } + assert(e.getCause.isInstanceOf[TimeoutException]) } private def testAsyncAction[R](action: RDD[Int] => FutureAction[R]): Unit = { @@ -221,7 +223,7 @@ class AsyncRDDActionsSuite extends SparkFunSuite with BeforeAndAfterAll with Tim // Now allow the executors to proceed with task processing. starter.release(rdd.partitions.length) // Waiting for the result verifies that the tasks were successfully processed. - Await.result(executionContextInvoked.future, atMost = 15.seconds) + ThreadUtils.awaitResult(executionContextInvoked.future, atMost = 15.seconds) } test("SimpleFutureAction callback must not consume a thread while waiting") { diff --git a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala index cebac2097f380..73803ec21a567 100644 --- a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala @@ -35,7 +35,7 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark.{SecurityManager, SparkConf, SparkEnv, SparkException, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} /** * Common tests for an RpcEnv implementation. @@ -415,7 +415,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { }) val f = endpointRef.ask[String]("Hi") - val ack = Await.result(f, 5 seconds) + val ack = ThreadUtils.awaitResult(f, 5 seconds) assert("ack" === ack) env.stop(endpointRef) @@ -435,7 +435,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "sendWithReply-remotely") try { val f = rpcEndpointRef.ask[String]("hello") - val ack = Await.result(f, 5 seconds) + val ack = ThreadUtils.awaitResult(f, 5 seconds) assert("ack" === ack) } finally { anotherEnv.shutdown() @@ -454,9 +454,9 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { val f = endpointRef.ask[String]("Hi") val e = intercept[SparkException] { - Await.result(f, 5 seconds) + ThreadUtils.awaitResult(f, 5 seconds) } - assert("Oops" === e.getMessage) + assert("Oops" === e.getCause.getMessage) env.stop(endpointRef) } @@ -476,9 +476,9 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { try { val f = rpcEndpointRef.ask[String]("hello") val e = intercept[SparkException] { - Await.result(f, 5 seconds) + ThreadUtils.awaitResult(f, 5 seconds) } - assert("Oops" === e.getMessage) + assert("Oops" === e.getCause.getMessage) } finally { anotherEnv.shutdown() anotherEnv.awaitTermination() @@ -487,6 +487,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { /** * Setup an [[RpcEndpoint]] to collect all network events. + * * @return the [[RpcEndpointRef]] and an `ConcurrentLinkedQueue` that contains network events. */ private def setupNetworkEndpoint( @@ -620,10 +621,10 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { anotherEnv.setupEndpointRef(env.address, "sendWithReply-unserializable-error") try { val f = rpcEndpointRef.ask[String]("hello") - val e = intercept[Exception] { - Await.result(f, 1 seconds) + val e = intercept[SparkException] { + ThreadUtils.awaitResult(f, 1 seconds) } - assert(e.isInstanceOf[NotSerializableException]) + assert(e.getCause.isInstanceOf[NotSerializableException]) } finally { anotherEnv.shutdown() anotherEnv.awaitTermination() @@ -754,15 +755,17 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { // RpcTimeout.awaitResult should have added the property to the TimeoutException message assert(reply2.contains(shortTimeout.timeoutProp)) - // Ask with delayed response and allow the Future to timeout before Await.result + // Ask with delayed response and allow the Future to timeout before ThreadUtils.awaitResult val fut3 = rpcEndpointRef.ask[String](NeverReply("goodbye"), shortTimeout) + // scalastyle:off awaitresult // Allow future to complete with failure using plain Await.result, this will return // once the future is complete to verify addMessageIfTimeout was invoked val reply3 = intercept[RpcTimeoutException] { Await.result(fut3, 2000 millis) }.getMessage + // scalastyle:on awaitresult // When the future timed out, the recover callback should have used // RpcTimeout.addMessageIfTimeout to add the property to the TimeoutException message diff --git a/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcEnvSuite.scala index 994a58836bd0d..2d6543d328618 100644 --- a/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcEnvSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcEnvSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.rpc.netty -import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark._ import org.apache.spark.rpc._ class NettyRpcEnvSuite extends RpcEnvSuite { @@ -34,10 +34,11 @@ class NettyRpcEnvSuite extends RpcEnvSuite { test("non-existent endpoint") { val uri = RpcEndpointAddress(env.address, "nonexist-endpoint").toString - val e = intercept[RpcEndpointNotFoundException] { + val e = intercept[SparkException] { env.setupEndpointRef(env.address, "nonexist-endpoint") } - assert(e.getMessage.contains(uri)) + assert(e.getCause.isInstanceOf[RpcEndpointNotFoundException]) + assert(e.getCause.getMessage.contains(uri)) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala index 8e509de7677c3..83288db92bb43 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.scheduler import java.io.File import java.util.concurrent.TimeoutException -import scala.concurrent.Await import scala.concurrent.duration._ import scala.language.postfixOps @@ -33,7 +32,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark._ import org.apache.spark.rdd.{FakeOutputCommitter, RDD} -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} /** * Unit tests for the output commit coordination functionality. @@ -159,9 +158,10 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { 0 until rdd.partitions.size, resultHandler, () => Unit) // It's an error if the job completes successfully even though no committer was authorized, // so throw an exception if the job was allowed to complete. - intercept[TimeoutException] { - Await.result(futureAction, 5 seconds) + val e = intercept[SparkException] { + ThreadUtils.awaitResult(futureAction, 5 seconds) } + assert(e.getCause.isInstanceOf[TimeoutException]) assert(tempDir.list().size === 0) } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala index 9d1bd7ec89bc7..9ee83b76e71dc 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala @@ -27,6 +27,7 @@ import org.scalatest.BeforeAndAfterEach import org.scalatest.time.SpanSugar._ import org.apache.spark.{SparkException, SparkFunSuite, TaskContext, TaskContextImpl} +import org.apache.spark.util.ThreadUtils class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { @@ -124,8 +125,8 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { } // After downgrading to a read lock, both threads should wake up and acquire the shared // read lock. - assert(!Await.result(lock1Future, 1.seconds)) - assert(!Await.result(lock2Future, 1.seconds)) + assert(!ThreadUtils.awaitResult(lock1Future, 1.seconds)) + assert(!ThreadUtils.awaitResult(lock2Future, 1.seconds)) assert(blockInfoManager.get("block").get.readerCount === 3) } @@ -161,7 +162,7 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { withTaskId(winningTID) { blockInfoManager.unlock("block") } - assert(!Await.result(losingFuture, 1.seconds)) + assert(!ThreadUtils.awaitResult(losingFuture, 1.seconds)) assert(blockInfoManager.get("block").get.readerCount === 1) } @@ -262,8 +263,8 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { withTaskId(0) { blockInfoManager.unlock("block") } - assert(Await.result(get1Future, 1.seconds).isDefined) - assert(Await.result(get2Future, 1.seconds).isDefined) + assert(ThreadUtils.awaitResult(get1Future, 1.seconds).isDefined) + assert(ThreadUtils.awaitResult(get2Future, 1.seconds).isDefined) assert(blockInfoManager.get("block").get.readerCount === 2) } @@ -288,13 +289,14 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { blockInfoManager.unlock("block") } assert( - Await.result(Future.firstCompletedOf(Seq(write1Future, write2Future)), 1.seconds).isDefined) + ThreadUtils.awaitResult( + Future.firstCompletedOf(Seq(write1Future, write2Future)), 1.seconds).isDefined) val firstWriteWinner = if (write1Future.isCompleted) 1 else 2 withTaskId(firstWriteWinner) { blockInfoManager.unlock("block") } - assert(Await.result(write1Future, 1.seconds).isDefined) - assert(Await.result(write2Future, 1.seconds).isDefined) + assert(ThreadUtils.awaitResult(write1Future, 1.seconds).isDefined) + assert(ThreadUtils.awaitResult(write2Future, 1.seconds).isDefined) } test("removing a non-existent block throws IllegalArgumentException") { @@ -344,8 +346,8 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { withTaskId(0) { blockInfoManager.removeBlock("block") } - assert(Await.result(getFuture, 1.seconds).isEmpty) - assert(Await.result(writeFuture, 1.seconds).isEmpty) + assert(ThreadUtils.awaitResult(getFuture, 1.seconds).isEmpty) + assert(ThreadUtils.awaitResult(writeFuture, 1.seconds).isEmpty) } test("releaseAllLocksForTask releases write locks") { diff --git a/core/src/test/scala/org/apache/spark/util/ThreadUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/ThreadUtilsSuite.scala index 6652a41b6990b..ae3b3d829f1bb 100644 --- a/core/src/test/scala/org/apache/spark/util/ThreadUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/ThreadUtilsSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.util import java.util.concurrent.{CountDownLatch, TimeUnit} -import scala.concurrent.{Await, Future} +import scala.concurrent.Future import scala.concurrent.duration._ import scala.util.Random @@ -109,7 +109,7 @@ class ThreadUtilsSuite extends SparkFunSuite { val f = Future { Thread.currentThread().getName() }(ThreadUtils.sameThread) - val futureThreadName = Await.result(f, 10.seconds) + val futureThreadName = ThreadUtils.awaitResult(f, 10.seconds) assert(futureThreadName === callerThreadName) } diff --git a/scalastyle-config.xml b/scalastyle-config.xml index a14e3e583f870..e39400e2d1840 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -192,6 +192,17 @@ This file is divided into 3 sections: ]]> + + Await\.result + + + JavaConversions diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala index 260dfb3f42244..94e676ded601b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.util.ThreadUtils /** * Additional tests for code generation. @@ -43,7 +44,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { } } - futures.foreach(Await.result(_, 10.seconds)) + futures.foreach(ThreadUtils.awaitResult(_, 10.seconds)) } test("SPARK-8443: split wide projections into blocks due to JVM code size limit") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 4091f65aecb50..415cd4d84a23f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicBoolean import scala.collection.mutable.ArrayBuffer import scala.concurrent.{Await, ExecutionContext, Future} import scala.concurrent.duration._ +import scala.util.control.NonFatal import org.apache.spark.{broadcast, SparkEnv} import org.apache.spark.internal.Logging @@ -167,7 +168,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ protected def waitForSubqueries(): Unit = { // fill in the result of subqueries subqueryResults.foreach { case (e, futureResult) => - val rows = Await.result(futureResult, Duration.Inf) + val rows = ThreadUtils.awaitResult(futureResult, Duration.Inf) if (rows.length > 1) { sys.error(s"more than one row returned by a subquery used as an expression:\n${e.plan}") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchange.scala index 102a9356df311..a4f42133425ee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchange.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.exchange -import scala.concurrent.{Await, ExecutionContext, Future} +import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration._ import org.apache.spark.broadcast @@ -81,8 +81,7 @@ case class BroadcastExchange( } override protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { - val result = Await.result(relationFuture, timeout) - result.asInstanceOf[broadcast.Broadcast[T]] + ThreadUtils.awaitResult(relationFuture, timeout).asInstanceOf[broadcast.Broadcast[T]] } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index eb49eabcb1ba9..0d0f556d9eae3 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -23,7 +23,7 @@ import java.sql.Timestamp import java.util.Date import scala.collection.mutable.ArrayBuffer -import scala.concurrent.{Await, Promise} +import scala.concurrent.Promise import scala.concurrent.duration._ import org.apache.hadoop.hive.conf.HiveConf.ConfVars @@ -32,7 +32,7 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.SparkFunSuite import org.apache.spark.internal.Logging import org.apache.spark.sql.test.ProcessTestUtils.ProcessOutputCapturer -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} /** * A test suite for the `spark-sql` CLI tool. Note that all test cases share the same temporary @@ -132,7 +132,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { new ProcessOutputCapturer(process.getErrorStream, captureOutput("stderr")).start() try { - Await.result(foundAllExpectedAnswers.future, timeout) + ThreadUtils.awaitResult(foundAllExpectedAnswers.future, timeout) } catch { case cause: Throwable => val message = s""" diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index a1268b8e94f56..ee14b6dc8d01d 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -24,7 +24,7 @@ import java.sql.{Date, DriverManager, SQLException, Statement} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import scala.concurrent.{Await, ExecutionContext, Future, Promise} +import scala.concurrent.{ExecutionContext, Future, Promise} import scala.concurrent.duration._ import scala.io.Source import scala.util.{Random, Try} @@ -40,7 +40,7 @@ import org.apache.thrift.protocol.TBinaryProtocol import org.apache.thrift.transport.TSocket import org.scalatest.BeforeAndAfterAll -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.internal.Logging import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.test.ProcessTestUtils.ProcessOutputCapturer @@ -373,9 +373,10 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { // slightly more conservatively than may be strictly necessary. Thread.sleep(1000) statement.cancel() - val e = intercept[SQLException] { - Await.result(f, 3.minute) - } + val e = intercept[SparkException] { + ThreadUtils.awaitResult(f, 3.minute) + }.getCause + assert(e.isInstanceOf[SQLException]) assert(e.getMessage.contains("cancelled")) // Cancellation is a no-op if spark.sql.hive.thriftServer.async=false @@ -391,7 +392,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { // might race and complete before we issue the cancel. Thread.sleep(1000) statement.cancel() - val rs1 = Await.result(sf, 3.minute) + val rs1 = ThreadUtils.awaitResult(sf, 3.minute) rs1.next() assert(rs1.getInt(1) === math.pow(5, 5)) rs1.close() @@ -814,7 +815,7 @@ abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAl process } - Await.result(serverStarted.future, SERVER_STARTUP_TIMEOUT) + ThreadUtils.awaitResult(serverStarted.future, SERVER_STARTUP_TIMEOUT) } private def stopThriftServer(): Unit = { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala index f381fa4094e74..a7d870500fd02 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.receiver -import scala.concurrent.{Await, ExecutionContext, Future} +import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration._ import scala.language.{existentials, postfixOps} import scala.util.control.NonFatal @@ -213,7 +213,7 @@ private[streaming] class WriteAheadLogBasedBlockHandler( // Combine the futures, wait for both to complete, and return the write ahead log record handle val combinedFuture = storeInBlockManagerFuture.zip(storeInWriteAheadLogFuture).map(_._2) - val walRecordHandle = Await.result(combinedFuture, blockStoreTimeout) + val walRecordHandle = ThreadUtils.awaitResult(combinedFuture, blockStoreTimeout) WriteAheadLogBasedStoreResult(blockId, numRecords, walRecordHandle) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala index 165e81ea41a98..71f3304f1ba73 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala @@ -23,14 +23,14 @@ import java.util.concurrent.LinkedBlockingQueue import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer -import scala.concurrent.{Await, Promise} +import scala.concurrent.Promise import scala.concurrent.duration._ import scala.util.control.NonFatal import org.apache.spark.SparkConf import org.apache.spark.internal.Logging import org.apache.spark.network.util.JavaUtils -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} /** * A wrapper for a WriteAheadLog that batches records before writing data. Handles aggregation @@ -80,7 +80,8 @@ private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog, conf: Sp } } if (putSuccessfully) { - Await.result(promise.future, WriteAheadLogUtils.getBatchingTimeout(conf).milliseconds) + ThreadUtils.awaitResult( + promise.future, WriteAheadLogUtils.getBatchingTimeout(conf).milliseconds) } else { throw new IllegalStateException("close() was called on BatchedWriteAheadLog before " + s"write request with time $time could be fulfilled.") diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index 8c980dee2cc06..24cb5afee33c2 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -38,7 +38,7 @@ import org.scalatest.concurrent.Eventually import org.scalatest.concurrent.Eventually._ import org.scalatest.mock.MockitoSugar -import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} import org.apache.spark.streaming.scheduler._ import org.apache.spark.util.{CompletionIterator, ManualClock, ThreadUtils, Utils} @@ -471,10 +471,11 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( // the BatchedWriteAheadLog should bubble up any exceptions that may have happened during writes val batchedWal = new BatchedWriteAheadLog(wal, sparkConf) - intercept[RuntimeException] { + val e = intercept[SparkException] { val buffer = mock[ByteBuffer] batchedWal.write(buffer, 2L) } + assert(e.getCause.getMessage === "Hello!") } // we make the write requests in separate threads so that we don't block the test thread From 5cb2e3360985bc9e67aee038befa93c258f2016a Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 19 Apr 2016 10:51:58 -0700 Subject: [PATCH 33/69] [SPARK-14675][SQL] ClassFormatError when use Seq as Aggregator buffer type ## What changes were proposed in this pull request? After https://github.com/apache/spark/pull/12067, we now use expressions to do the aggregation in `TypedAggregateExpression`. To implement buffer merge, we produce a new buffer deserializer expression by replacing `AttributeReference` with right-side buffer attribute, like other `DeclarativeAggregate`s do, and finally combine the left and right buffer deserializer with `Invoke`. However, after https://github.com/apache/spark/pull/12338, we will add loop variable to class members when codegen `MapObjects`. If the `Aggregator` buffer type is `Seq`, which is implemented by `MapObjects` expression, we will add the same loop variable to class members twice(by left and right buffer deserializer), which cause the `ClassFormatError`. This PR fixes this issue by calling `distinct` before declare the class menbers. ## How was this patch tested? new regression test in `DatasetAggregatorSuite` Author: Wenchen Fan Closes #12468 from cloud-fan/bug. --- .../expressions/EquivalentExpressions.scala | 13 ++++++++++-- .../expressions/codegen/CodeGenerator.scala | 8 ++++++-- .../spark/sql/DatasetAggregatorSuite.scala | 20 +++++++++++++++++++ 3 files changed, 37 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala index 8d8cc152ff29c..607c7c877cc14 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala @@ -69,8 +69,17 @@ class EquivalentExpressions { */ def addExprTree(root: Expression, ignoreLeaf: Boolean = true): Unit = { val skip = root.isInstanceOf[LeafExpression] && ignoreLeaf - // the children of CodegenFallback will not be used to generate code (call eval() instead) - if (!skip && !addExpr(root) && !root.isInstanceOf[CodegenFallback]) { + // There are some special expressions that we should not recurse into children. + // 1. CodegenFallback: it's children will not be used to generate code (call eval() instead) + // 2. ReferenceToExpressions: it's kind of an explicit sub-expression elimination. + val shouldRecurse = root match { + // TODO: some expressions implements `CodegenFallback` but can still do codegen, + // e.g. `CaseWhen`, we should support them. + case _: CodegenFallback => false + case _: ReferenceToExpressions => false + case _ => true + } + if (!skip && !addExpr(root) && shouldRecurse) { root.children.foreach(addExprTree(_, ignoreLeaf)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 38ac13b208aab..d29c27c14b0c3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -110,13 +110,17 @@ class CodegenContext { } def declareMutableStates(): String = { - mutableStates.map { case (javaType, variableName, _) => + // It's possible that we add same mutable state twice, e.g. the `mergeExpressions` in + // `TypedAggregateExpression`, we should call `distinct` here to remove the duplicated ones. + mutableStates.distinct.map { case (javaType, variableName, _) => s"private $javaType $variableName;" }.mkString("\n") } def initMutableStates(): String = { - mutableStates.map(_._3).mkString("\n") + // It's possible that we add same mutable state twice, e.g. the `mergeExpressions` in + // `TypedAggregateExpression`, we should call `distinct` here to remove the duplicated ones. + mutableStates.distinct.map(_._3).mkString("\n") } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala index 0d84a594f71a9..6eae3ed7ad6c0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql import scala.language.postfixOps +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.expressions.Aggregator import org.apache.spark.sql.expressions.scala.typed import org.apache.spark.sql.functions._ @@ -72,6 +73,16 @@ object NameAgg extends Aggregator[AggData, String, String] { } +object SeqAgg extends Aggregator[AggData, Seq[Int], Seq[Int]] { + def zero: Seq[Int] = Nil + def reduce(b: Seq[Int], a: AggData): Seq[Int] = a.a +: b + def merge(b1: Seq[Int], b2: Seq[Int]): Seq[Int] = b1 ++ b2 + def finish(r: Seq[Int]): Seq[Int] = r + override def bufferEncoder: Encoder[Seq[Int]] = ExpressionEncoder() + override def outputEncoder: Encoder[Seq[Int]] = ExpressionEncoder() +} + + class ParameterizedTypeSum[IN, OUT : Numeric : Encoder](f: IN => OUT) extends Aggregator[IN, OUT, OUT] { @@ -212,4 +223,13 @@ class DatasetAggregatorSuite extends QueryTest with SharedSQLContext { val df = Seq(1 -> "a", 2 -> "b", 3 -> "b").toDF("i", "j") checkAnswer(df.groupBy($"j").agg(RowAgg.toColumn), Row("a", 1) :: Row("b", 5) :: Nil) } + + test("SPARK-14675: ClassFormatError when use Seq as Aggregator buffer type") { + val ds = Seq(AggData(1, "a"), AggData(2, "a")).toDS() + + checkDataset( + ds.groupByKey(_.b).agg(SeqAgg.toColumn), + "a" -> Seq(1, 2) + ) + } } From 0b8369d8548c0204b9c24d826c731063b72360b8 Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 19 Apr 2016 23:30:34 +0200 Subject: [PATCH 34/69] [SPARK-14656][CORE] Fix Benchmark.getPorcessorName() always return "Unknown processor" on Linux ## What changes were proposed in this pull request? This PR returns correct processor name in ```/proc/cpuinfo``` on Linux from ```Benchmark.getPorcessorName()```. Now, this return ```Unknown processor```. Since ```Utils.executeAndGetOutput(Seq("which", "grep"))``` return ```/bin/grep\n```, it is failed to execute ```/bin/grep\n```. This PR strips ```\n``` at the end of the line of a result of ```Utils.executeAndGetOutput()``` Before applying this PR ```` Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Linux 2.6.32-504.el6.x86_64 Unknown processor back-to-back filter: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------- Dataset 472 / 503 21.2 47.2 1.0X DataFrame 51 / 58 198.0 5.1 9.3X RDD 189 / 211 52.8 18.9 2.5X ```` After applying this PR ``` Java HotSpot(TM) 64-Bit Server VM 1.8.0_66-b17 on Linux 2.6.32-504.el6.x86_64 Intel(R) Xeon(R) CPU E5-2667 v2 3.30GHz back-to-back filter: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------- Dataset 490 / 502 20.4 49.0 1.0X DataFrame 55 / 61 183.4 5.5 9.0X RDD 210 / 237 47.7 21.0 2.3X ``` ## How was this patch tested? Run Benchmark programs on Linux by hand Author: Kazuaki Ishizaki Closes #12411 from kiszk/SPARK-14656. --- core/src/main/scala/org/apache/spark/util/Benchmark.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Benchmark.scala b/core/src/main/scala/org/apache/spark/util/Benchmark.scala index 9e40bafd521d7..3718542810d0b 100644 --- a/core/src/main/scala/org/apache/spark/util/Benchmark.scala +++ b/core/src/main/scala/org/apache/spark/util/Benchmark.scala @@ -96,9 +96,9 @@ private[spark] object Benchmark { Utils.executeAndGetOutput(Seq("/usr/sbin/sysctl", "-n", "machdep.cpu.brand_string")) } else if (SystemUtils.IS_OS_LINUX) { Try { - val grepPath = Utils.executeAndGetOutput(Seq("which", "grep")) + val grepPath = Utils.executeAndGetOutput(Seq("which", "grep")).stripLineEnd Utils.executeAndGetOutput(Seq(grepPath, "-m", "1", "model name", "/proc/cpuinfo")) - .replaceFirst("model name[\\s*]:[\\s*]", "") + .stripLineEnd.replaceFirst("model name[\\s*]:[\\s*]", "") }.getOrElse("Unknown processor") } else { System.getenv("PROCESSOR_IDENTIFIER") From 3c91afec20607e0d853433a904105ee22df73c73 Mon Sep 17 00:00:00 2001 From: Nezih Yigitbasi Date: Tue, 19 Apr 2016 14:35:26 -0700 Subject: [PATCH 35/69] [SPARK-14042][CORE] Add custom coalescer support ## What changes were proposed in this pull request? This PR adds support for specifying an optional custom coalescer to the `coalesce()` method. Currently I have only added this feature to the `RDD` interface, and once we sort out the details we can proceed with adding this feature to the other APIs (`Dataset` etc.) ## How was this patch tested? Added a unit test for this functionality. /cc rxin (per our discussion on the mailing list) Author: Nezih Yigitbasi Closes #11865 from nezihyigitbasi/custom_coalesce_policy. --- .../org/apache/spark/rdd/CoalescedRDD.scala | 99 +++++++++---------- .../main/scala/org/apache/spark/rdd/RDD.scala | 9 +- .../apache/spark/rdd/coalesce-public.scala | 52 ++++++++++ .../scala/org/apache/spark/rdd/RDDSuite.scala | 99 ++++++++++++++++++- project/MimaExcludes.scala | 4 + 5 files changed, 209 insertions(+), 54 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/rdd/coalesce-public.scala diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index 35665ab7c030c..e75f1dbf8107a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -70,23 +70,23 @@ private[spark] case class CoalescedRDDPartition( * parent partitions * @param prev RDD to be coalesced * @param maxPartitions number of desired partitions in the coalesced RDD (must be positive) - * @param balanceSlack used to trade-off balance and locality. 1.0 is all locality, 0 is all balance + * @param partitionCoalescer [[PartitionCoalescer]] implementation to use for coalescing */ private[spark] class CoalescedRDD[T: ClassTag]( @transient var prev: RDD[T], maxPartitions: Int, - balanceSlack: Double = 0.10) + partitionCoalescer: Option[PartitionCoalescer] = None) extends RDD[T](prev.context, Nil) { // Nil since we implement getDependencies require(maxPartitions > 0 || maxPartitions == prev.partitions.length, s"Number of partitions ($maxPartitions) must be positive.") override def getPartitions: Array[Partition] = { - val pc = new PartitionCoalescer(maxPartitions, prev, balanceSlack) + val pc = partitionCoalescer.getOrElse(new DefaultPartitionCoalescer()) - pc.run().zipWithIndex.map { + pc.coalesce(maxPartitions, prev).zipWithIndex.map { case (pg, i) => - val ids = pg.arr.map(_.index).toArray + val ids = pg.partitions.map(_.index).toArray new CoalescedRDDPartition(i, prev, ids, pg.prefLoc) } } @@ -144,15 +144,15 @@ private[spark] class CoalescedRDD[T: ClassTag]( * desired partitions is greater than the number of preferred machines (can happen), it needs to * start picking duplicate preferred machines. This is determined using coupon collector estimation * (2n log(n)). The load balancing is done using power-of-two randomized bins-balls with one twist: - * it tries to also achieve locality. This is done by allowing a slack (balanceSlack) between two - * bins. If two bins are within the slack in terms of balance, the algorithm will assign partitions - * according to locality. (contact alig for questions) - * + * it tries to also achieve locality. This is done by allowing a slack (balanceSlack, where + * 1.0 is all locality, 0 is all balance) between two bins. If two bins are within the slack + * in terms of balance, the algorithm will assign partitions according to locality. + * (contact alig for questions) */ -private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) { - - def compare(o1: PartitionGroup, o2: PartitionGroup): Boolean = o1.size < o2.size +private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) + extends PartitionCoalescer { + def compare(o1: PartitionGroup, o2: PartitionGroup): Boolean = o1.numPartitions < o2.numPartitions def compare(o1: Option[PartitionGroup], o2: Option[PartitionGroup]): Boolean = if (o1 == None) false else if (o2 == None) true else compare(o1.get, o2.get) @@ -167,14 +167,10 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: // hash used for the first maxPartitions (to avoid duplicates) val initialHash = mutable.Set[Partition]() - // determines the tradeoff between load-balancing the partitions sizes and their locality - // e.g. balanceSlack=0.10 means that it allows up to 10% imbalance in favor of locality - val slack = (balanceSlack * prev.partitions.length).toInt - var noLocality = true // if true if no preferredLocations exists for parent RDD // gets the *current* preferred locations from the DAGScheduler (as opposed to the static ones) - def currPrefLocs(part: Partition): Seq[String] = { + def currPrefLocs(part: Partition, prev: RDD[_]): Seq[String] = { prev.context.getPreferredLocs(prev, part.index).map(tl => tl.host) } @@ -192,7 +188,7 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: def resetIterator(): Iterator[(String, Partition)] = { val iterators = (0 to 2).map { x => prev.partitions.iterator.flatMap { p => - if (currPrefLocs(p).size > x) Some((currPrefLocs(p)(x), p)) else None + if (currPrefLocs(p, prev).size > x) Some((currPrefLocs(p, prev)(x), p)) else None } } iterators.reduceLeft((x, y) => x ++ y) @@ -215,8 +211,9 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: /** * Sorts and gets the least element of the list associated with key in groupHash * The returned PartitionGroup is the least loaded of all groups that represent the machine "key" + * * @param key string representing a partitioned group on preferred machine key - * @return Option of PartitionGroup that has least elements for key + * @return Option of [[PartitionGroup]] that has least elements for key */ def getLeastGroupHash(key: String): Option[PartitionGroup] = { groupHash.get(key).map(_.sortWith(compare).head) @@ -224,7 +221,7 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: def addPartToPGroup(part: Partition, pgroup: PartitionGroup): Boolean = { if (!initialHash.contains(part)) { - pgroup.arr += part // already assign this element + pgroup.partitions += part // already assign this element initialHash += part // needed to avoid assigning partitions to multiple buckets true } else { false } @@ -236,12 +233,12 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: * until it has seen most of the preferred locations (2 * n log(n)) * @param targetLen */ - def setupGroups(targetLen: Int) { + def setupGroups(targetLen: Int, prev: RDD[_]) { val rotIt = new LocationIterator(prev) // deal with empty case, just create targetLen partition groups with no preferred location if (!rotIt.hasNext) { - (1 to targetLen).foreach(x => groupArr += PartitionGroup()) + (1 to targetLen).foreach(x => groupArr += new PartitionGroup()) return } @@ -259,7 +256,7 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: tries += 1 val (nxt_replica, nxt_part) = rotIt.next() if (!groupHash.contains(nxt_replica)) { - val pgroup = PartitionGroup(nxt_replica) + val pgroup = new PartitionGroup(Some(nxt_replica)) groupArr += pgroup addPartToPGroup(nxt_part, pgroup) groupHash.put(nxt_replica, ArrayBuffer(pgroup)) // list in case we have multiple @@ -269,7 +266,7 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: while (numCreated < targetLen) { // if we don't have enough partition groups, create duplicates var (nxt_replica, nxt_part) = rotIt.next() - val pgroup = PartitionGroup(nxt_replica) + val pgroup = new PartitionGroup(Some(nxt_replica)) groupArr += pgroup groupHash.getOrElseUpdate(nxt_replica, ArrayBuffer()) += pgroup var tries = 0 @@ -285,17 +282,29 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: /** * Takes a parent RDD partition and decides which of the partition groups to put it in * Takes locality into account, but also uses power of 2 choices to load balance - * It strikes a balance between the two use the balanceSlack variable + * It strikes a balance between the two using the balanceSlack variable * @param p partition (ball to be thrown) + * @param balanceSlack determines the trade-off between load-balancing the partitions sizes and + * their locality. e.g., balanceSlack=0.10 means that it allows up to 10% + * imbalance in favor of locality * @return partition group (bin to be put in) */ - def pickBin(p: Partition): PartitionGroup = { - val pref = currPrefLocs(p).map(getLeastGroupHash(_)).sortWith(compare) // least loaded pref locs + def pickBin(p: Partition, prev: RDD[_], balanceSlack: Double): PartitionGroup = { + val slack = (balanceSlack * prev.partitions.length).toInt + // least loaded pref locs + val pref = currPrefLocs(p, prev).map(getLeastGroupHash(_)).sortWith(compare) val prefPart = if (pref == Nil) None else pref.head val r1 = rnd.nextInt(groupArr.size) val r2 = rnd.nextInt(groupArr.size) - val minPowerOfTwo = if (groupArr(r1).size < groupArr(r2).size) groupArr(r1) else groupArr(r2) + val minPowerOfTwo = { + if (groupArr(r1).numPartitions < groupArr(r2).numPartitions) { + groupArr(r1) + } + else { + groupArr(r2) + } + } if (prefPart.isEmpty) { // if no preferred locations, just use basic power of two return minPowerOfTwo @@ -303,55 +312,45 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: val prefPartActual = prefPart.get - if (minPowerOfTwo.size + slack <= prefPartActual.size) { // more imbalance than the slack allows + // more imbalance than the slack allows + if (minPowerOfTwo.numPartitions + slack <= prefPartActual.numPartitions) { minPowerOfTwo // prefer balance over locality } else { prefPartActual // prefer locality over balance } } - def throwBalls() { + def throwBalls(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) { if (noLocality) { // no preferredLocations in parent RDD, no randomization needed if (maxPartitions > groupArr.size) { // just return prev.partitions for ((p, i) <- prev.partitions.zipWithIndex) { - groupArr(i).arr += p + groupArr(i).partitions += p } } else { // no locality available, then simply split partitions based on positions in array for (i <- 0 until maxPartitions) { val rangeStart = ((i.toLong * prev.partitions.length) / maxPartitions).toInt val rangeEnd = (((i.toLong + 1) * prev.partitions.length) / maxPartitions).toInt - (rangeStart until rangeEnd).foreach{ j => groupArr(i).arr += prev.partitions(j) } + (rangeStart until rangeEnd).foreach{ j => groupArr(i).partitions += prev.partitions(j) } } } } else { for (p <- prev.partitions if (!initialHash.contains(p))) { // throw every partition into group - pickBin(p).arr += p + pickBin(p, prev, balanceSlack).partitions += p } } } - def getPartitions: Array[PartitionGroup] = groupArr.filter( pg => pg.size > 0).toArray + def getPartitions: Array[PartitionGroup] = groupArr.filter( pg => pg.numPartitions > 0).toArray /** * Runs the packing algorithm and returns an array of PartitionGroups that if possible are * load balanced and grouped by locality - * @return array of partition groups + * + * @return array of partition groups */ - def run(): Array[PartitionGroup] = { - setupGroups(math.min(prev.partitions.length, maxPartitions)) // setup the groups (bins) - throwBalls() // assign partitions (balls) to each group (bins) + def coalesce(maxPartitions: Int, prev: RDD[_]): Array[PartitionGroup] = { + setupGroups(math.min(prev.partitions.length, maxPartitions), prev) // setup the groups (bins) + throwBalls(maxPartitions, prev, balanceSlack) // assign partitions (balls) to each group (bins) getPartitions } } - -private case class PartitionGroup(prefLoc: Option[String] = None) { - var arr = mutable.ArrayBuffer[Partition]() - def size: Int = arr.size -} - -private object PartitionGroup { - def apply(prefLoc: String): PartitionGroup = { - require(prefLoc != "", "Preferred location must not be empty") - PartitionGroup(Some(prefLoc)) - } -} diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index f6e0148f78333..499a8b9aa1a89 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -433,7 +433,9 @@ abstract class RDD[T: ClassTag]( * coalesce(1000, shuffle = true) will result in 1000 partitions with the * data distributed using a hash partitioner. */ - def coalesce(numPartitions: Int, shuffle: Boolean = false)(implicit ord: Ordering[T] = null) + def coalesce(numPartitions: Int, shuffle: Boolean = false, + partitionCoalescer: Option[PartitionCoalescer] = Option.empty) + (implicit ord: Ordering[T] = null) : RDD[T] = withScope { if (shuffle) { /** Distributes elements evenly across output partitions, starting from a random partition. */ @@ -451,9 +453,10 @@ abstract class RDD[T: ClassTag]( new CoalescedRDD( new ShuffledRDD[Int, T, T](mapPartitionsWithIndex(distributePartition), new HashPartitioner(numPartitions)), - numPartitions).values + numPartitions, + partitionCoalescer).values } else { - new CoalescedRDD(this, numPartitions) + new CoalescedRDD(this, numPartitions, partitionCoalescer) } } diff --git a/core/src/main/scala/org/apache/spark/rdd/coalesce-public.scala b/core/src/main/scala/org/apache/spark/rdd/coalesce-public.scala new file mode 100644 index 0000000000000..d8a80aa5aeb15 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rdd/coalesce-public.scala @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.rdd + +import scala.collection.mutable + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.Partition + +/** + * ::DeveloperApi:: + * A PartitionCoalescer defines how to coalesce the partitions of a given RDD. + */ +@DeveloperApi +trait PartitionCoalescer { + + /** + * Coalesce the partitions of the given RDD. + * + * @param maxPartitions the maximum number of partitions to have after coalescing + * @param parent the parent RDD whose partitions to coalesce + * @return an array of [[PartitionGroup]]s, where each element is itself an array of + * [[Partition]]s and represents a partition after coalescing is performed. + */ + def coalesce(maxPartitions: Int, parent: RDD[_]): Array[PartitionGroup] +} + +/** + * ::DeveloperApi:: + * A group of [[Partition]]s + * @param prefLoc preferred location for the partition group + */ +@DeveloperApi +class PartitionGroup(val prefLoc: Option[String] = None) { + val partitions = mutable.ArrayBuffer[Partition]() + def numPartitions: Int = partitions.size +} diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 24daedab2090f..8dc463d56d182 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -17,13 +17,15 @@ package org.apache.spark.rdd -import java.io.{IOException, ObjectInputStream, ObjectOutputStream} +import java.io.{File, IOException, ObjectInputStream, ObjectOutputStream} import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.reflect.ClassTag import com.esotericsoftware.kryo.KryoException +import org.apache.hadoop.io.{LongWritable, Text} +import org.apache.hadoop.mapred.{FileSplit, TextInputFormat} import org.apache.spark._ import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} @@ -31,6 +33,20 @@ import org.apache.spark.rdd.RDDSuiteUtils._ import org.apache.spark.util.Utils class RDDSuite extends SparkFunSuite with SharedSparkContext { + var tempDir: File = _ + + override def beforeAll(): Unit = { + super.beforeAll() + tempDir = Utils.createTempDir() + } + + override def afterAll(): Unit = { + try { + Utils.deleteRecursively(tempDir) + } finally { + super.afterAll() + } + } test("basic operations") { val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) @@ -951,6 +967,32 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { assert(thrown.getMessage.contains("SPARK-5063")) } + test("custom RDD coalescer") { + val maxSplitSize = 512 + val outDir = new File(tempDir, "output").getAbsolutePath + sc.makeRDD(1 to 1000, 10).saveAsTextFile(outDir) + val hadoopRDD = + sc.hadoopFile(outDir, classOf[TextInputFormat], classOf[LongWritable], classOf[Text]) + val coalescedHadoopRDD = + hadoopRDD.coalesce(2, partitionCoalescer = Option(new SizeBasedCoalescer(maxSplitSize))) + assert(coalescedHadoopRDD.partitions.size <= 10) + var totalPartitionCount = 0L + coalescedHadoopRDD.partitions.foreach(partition => { + var splitSizeSum = 0L + partition.asInstanceOf[CoalescedRDDPartition].parents.foreach(partition => { + val split = partition.asInstanceOf[HadoopPartition].inputSplit.value.asInstanceOf[FileSplit] + splitSizeSum += split.getLength + totalPartitionCount += 1 + }) + assert(splitSizeSum <= maxSplitSize) + }) + assert(totalPartitionCount == 10) + } + + // NOTE + // Below tests calling sc.stop() have to be the last tests in this suite. If there are tests + // running after them and if they access sc those tests will fail as sc is already closed, because + // sc is shared (this suite mixins SharedSparkContext) test("cannot run actions after SparkContext has been stopped (SPARK-5063)") { val existingRDD = sc.parallelize(1 to 100) sc.stop() @@ -971,5 +1013,60 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { assertFails { sc.parallelize(1 to 100) } assertFails { sc.textFile("/nonexistent-path") } } +} +/** + * Coalesces partitions based on their size assuming that the parent RDD is a [[HadoopRDD]]. + * Took this class out of the test suite to prevent "Task not serializable" exceptions. + */ +class SizeBasedCoalescer(val maxSize: Int) extends PartitionCoalescer with Serializable { + override def coalesce(maxPartitions: Int, parent: RDD[_]): Array[PartitionGroup] = { + val partitions: Array[Partition] = parent.asInstanceOf[HadoopRDD[Any, Any]].getPartitions + val groups = ArrayBuffer[PartitionGroup]() + var currentGroup = new PartitionGroup() + var currentSum = 0L + var totalSum = 0L + var index = 0 + + // sort partitions based on the size of the corresponding input splits + partitions.sortWith((partition1, partition2) => { + val partition1Size = partition1.asInstanceOf[HadoopPartition].inputSplit.value.getLength + val partition2Size = partition2.asInstanceOf[HadoopPartition].inputSplit.value.getLength + partition1Size < partition2Size + }) + + def updateGroups(): Unit = { + groups += currentGroup + currentGroup = new PartitionGroup() + currentSum = 0 + } + + def addPartition(partition: Partition, splitSize: Long): Unit = { + currentGroup.partitions += partition + currentSum += splitSize + totalSum += splitSize + } + + while (index < partitions.size) { + val partition = partitions(index) + val fileSplit = + partition.asInstanceOf[HadoopPartition].inputSplit.value.asInstanceOf[FileSplit] + val splitSize = fileSplit.getLength + if (currentSum + splitSize < maxSize) { + addPartition(partition, splitSize) + index += 1 + if (index == partitions.size) { + updateGroups + } + } else { + if (currentGroup.partitions.size == 0) { + addPartition(partition, splitSize) + index += 1 + } else { + updateGroups + } + } + } + groups.toArray + } } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index ff35dc010dc2f..b2c80afb533c1 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -49,6 +49,10 @@ object MimaExcludes { "org.apache.spark.status.api.v1.ApplicationAttemptInfo.this"), ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.status.api.v1.ApplicationAttemptInfo.$default$5"), + // SPARK-14042 Add custom coalescer support + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.rdd.RDD.coalesce"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.rdd.PartitionCoalescer$LocationIterator"), + ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.rdd.PartitionCoalescer"), // SPARK-12600 Remove SQL deprecated methods ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLContext$QueryExecution"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.SQLContext$SparkPlanner"), From da8859226e09aa6ebcf6a1c5c1369dec3c216eac Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Tue, 19 Apr 2016 15:16:02 -0700 Subject: [PATCH 36/69] [SPARK-4226] [SQL] Support IN/EXISTS Subqueries ### What changes were proposed in this pull request? This PR adds support for in/exists predicate subqueries to Spark. Predicate sub-queries are used as a filtering condition in a query (this is the only supported use case). A predicate sub-query comes in two forms: - `[NOT] EXISTS(subquery)` - `[NOT] IN (subquery)` This PR is (loosely) based on the work of davies (https://github.com/apache/spark/pull/10706) and chenghao-intel (https://github.com/apache/spark/pull/9055). They should be credited for the work they did. ### How was this patch tested? Modified parsing unit tests. Added tests to `org.apache.spark.sql.SQLQuerySuite` cc rxin, davies & chenghao-intel Author: Herman van Hovell Closes #12306 from hvanhovell/SPARK-4226. --- .../sql/catalyst/analysis/Analyzer.scala | 30 +++-- .../sql/catalyst/analysis/CheckAnalysis.scala | 40 +++++- .../sql/catalyst/expressions/subquery.scala | 84 ++++++++++++- .../sql/catalyst/optimizer/Optimizer.scala | 115 +++++++++++++++++- .../sql/catalyst/parser/AstBuilder.scala | 16 ++- .../analysis/AnalysisErrorSuite.scala | 58 ++++++++- .../catalyst/parser/ErrorParserSuite.scala | 6 +- .../parser/ExpressionParserSuite.scala | 8 +- .../sql/catalyst/parser/PlanParserSuite.scala | 4 +- .../apache/spark/sql/execution/subquery.scala | 6 +- .../org/apache/spark/sql/QueryTest.scala | 53 ++++++-- .../org/apache/spark/sql/SubquerySuite.scala | 98 +++++++++++++++ 12 files changed, 476 insertions(+), 42 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 0e2fd43983ef1..236476900a519 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -855,25 +855,35 @@ class Analyzer( } /** - * This rule resolve subqueries inside expressions. + * This rule resolves sub-queries inside expressions. * - * Note: CTE are handled in CTESubstitution. + * Note: CTEs are handled in CTESubstitution. */ object ResolveSubquery extends Rule[LogicalPlan] with PredicateHelper { - private def hasSubquery(e: Expression): Boolean = { - e.find(_.isInstanceOf[SubqueryExpression]).isDefined - } - - private def hasSubquery(q: LogicalPlan): Boolean = { - q.expressions.exists(hasSubquery) + /** + * Resolve the correlated predicates in the [[Filter]] clauses (e.g. WHERE or HAVING) of a + * sub-query by using the plan the predicates should be correlated to. + */ + private def resolveCorrelatedPredicates(q: LogicalPlan, p: LogicalPlan): LogicalPlan = { + q transformUp { + case f @ Filter(cond, child) if child.resolved && !f.resolved => + val newCond = resolveExpression(cond, p, throws = false) + if (!cond.fastEquals(newCond)) { + Filter(newCond, child) + } else { + f + } + } } def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case q: LogicalPlan if q.childrenResolved && hasSubquery(q) => + case q: LogicalPlan if q.childrenResolved => q transformExpressions { case e: SubqueryExpression if !e.query.resolved => - e.withNewPlan(execute(e.query)) + // First resolve as much of the sub-query as possible. After that we use the children of + // this plan to resolve the remaining correlated predicates. + e.withNewPlan(q.children.foldLeft(execute(e.query))(resolveCorrelatedPredicates)) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index d6a8c3eec81aa..45e4d535c18cc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -20,14 +20,14 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression -import org.apache.spark.sql.catalyst.plans.UsingJoin +import org.apache.spark.sql.catalyst.plans.{Inner, RightOuter, UsingJoin} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.types._ /** * Throws user facing errors when passed invalid queries that fail to analyze. */ -trait CheckAnalysis { +trait CheckAnalysis extends PredicateHelper { /** * Override to provide additional checks for correct analysis. @@ -110,6 +110,39 @@ trait CheckAnalysis { s"filter expression '${f.condition.sql}' " + s"of type ${f.condition.dataType.simpleString} is not a boolean.") + case f @ Filter(condition, child) => + // Make sure that no correlated reference is below Aggregates, Outer Joins and on the + // right hand side of Unions. + lazy val attributes = child.outputSet + def failOnCorrelatedReference( + p: LogicalPlan, + message: String): Unit = p.transformAllExpressions { + case e: NamedExpression if attributes.contains(e) => + failAnalysis(s"Accessing outer query column is not allowed in $message: $e") + } + def checkForCorrelatedReferences(p: PredicateSubquery): Unit = p.query.foreach { + case a @ Aggregate(_, _, source) => + failOnCorrelatedReference(source, "an AGGREATE") + case j @ Join(left, _, RightOuter, _) => + failOnCorrelatedReference(left, "a RIGHT OUTER JOIN") + case j @ Join(_, right, jt, _) if jt != Inner => + failOnCorrelatedReference(right, "a LEFT (OUTER) JOIN") + case Union(_ :: xs) => + xs.foreach(failOnCorrelatedReference(_, "a UNION")) + case s: SetOperation => + failOnCorrelatedReference(s.right, "an INTERSECT/EXCEPT") + case _ => + } + splitConjunctivePredicates(condition).foreach { + case p: PredicateSubquery => + checkForCorrelatedReferences(p) + case Not(p: PredicateSubquery) => + checkForCorrelatedReferences(p) + case e if PredicateSubquery.hasPredicateSubquery(e) => + failAnalysis(s"Predicate sub-queries cannot be used in nested conditions: $e") + case e => + } + case j @ Join(_, _, UsingJoin(_, cols), _) => val from = operator.inputSet.map(_.name).mkString(", ") failAnalysis( @@ -209,6 +242,9 @@ trait CheckAnalysis { | but one table has '${firstError.output.length}' columns and another table has | '${s.children.head.output.length}' columns""".stripMargin) + case p if p.expressions.exists(PredicateSubquery.hasPredicateSubquery) => + failAnalysis(s"Predicate sub-queries can only be used in a Filter: $p") + case _ => // Fallbacks to the following checks } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala index 968bbdb1a5f03..cbee0e61f7a7a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/subquery.scala @@ -20,12 +20,12 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} -import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.types._ /** * An interface for subquery that is used in expressions. */ -abstract class SubqueryExpression extends LeafExpression { +abstract class SubqueryExpression extends Expression { /** * The logical plan of the query. @@ -61,6 +61,8 @@ case class ScalarSubquery( override def dataType: DataType = query.schema.fields.head.dataType + override def children: Seq[Expression] = Nil + override def checkInputDataTypes(): TypeCheckResult = { if (query.schema.length != 1) { TypeCheckResult.TypeCheckFailure("Scalar subquery must return only one column, but got " + @@ -77,3 +79,81 @@ case class ScalarSubquery( override def toString: String = s"subquery#${exprId.id}" } + +/** + * A predicate subquery checks the existence of a value in a sub-query. We currently only allow + * [[PredicateSubquery]] expressions within a Filter plan (i.e. WHERE or a HAVING clause). This will + * be rewritten into a left semi/anti join during analysis. + */ +abstract class PredicateSubquery extends SubqueryExpression with Unevaluable with Predicate { + override def nullable: Boolean = false + override def plan: LogicalPlan = SubqueryAlias(prettyName, query) +} + +object PredicateSubquery { + def hasPredicateSubquery(e: Expression): Boolean = { + e.find(_.isInstanceOf[PredicateSubquery]).isDefined + } +} + +/** + * The [[InSubQuery]] predicate checks the existence of a value in a sub-query. For example (SQL): + * {{{ + * SELECT * + * FROM a + * WHERE a.id IN (SELECT id + * FROM b) + * }}} + */ +case class InSubQuery(value: Expression, query: LogicalPlan) extends PredicateSubquery { + override def children: Seq[Expression] = value :: Nil + override lazy val resolved: Boolean = value.resolved && query.resolved + override def withNewPlan(plan: LogicalPlan): InSubQuery = InSubQuery(value, plan) + + /** + * The unwrapped value side expressions. + */ + lazy val expressions: Seq[Expression] = value match { + case CreateStruct(cols) => cols + case col => Seq(col) + } + + /** + * Check if the number of columns and the data types on both sides match. + */ + override def checkInputDataTypes(): TypeCheckResult = { + // Check the number of arguments. + if (expressions.length != query.output.length) { + TypeCheckResult.TypeCheckFailure( + s"The number of fields in the value (${expressions.length}) does not match with " + + s"the number of columns in the subquery (${query.output.length})") + } + + // Check the argument types. + expressions.zip(query.output).zipWithIndex.foreach { + case ((e, a), i) if e.dataType != a.dataType => + TypeCheckResult.TypeCheckFailure( + s"The data type of value[$i](${e.dataType}) does not match " + + s"subquery column '${a.name}' (${a.dataType}).") + case _ => + } + + TypeCheckResult.TypeCheckSuccess + } +} + +/** + * The [[Exists]] expression checks if a row exists in a subquery given some correlated condition. + * For example (SQL): + * {{{ + * SELECT * + * FROM a + * WHERE EXISTS (SELECT * + * FROM b + * WHERE b.id = a.id) + * }}} + */ +case class Exists(query: LogicalPlan) extends PredicateSubquery { + override def children: Seq[Expression] = Nil + override def withNewPlan(plan: LogicalPlan): Exists = Exists(plan) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 0a5232b2d485d..ecc2d773e7753 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -19,11 +19,12 @@ package org.apache.spark.sql.catalyst.optimizer import scala.annotation.tailrec import scala.collection.immutable.HashSet +import scala.collection.mutable import org.apache.spark.sql.catalyst.{CatalystConf, SimpleCatalystConf} import org.apache.spark.sql.catalyst.analysis.{CleanupAliases, DistinctAggregationRewriter, EliminateSubqueryAliases, EmptyFunctionRegistry} import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} -import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.{InSubQuery, _} import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral} import org.apache.spark.sql.catalyst.planning.{ExtractFiltersAndInnerJoins, Unions} @@ -47,6 +48,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) // However, because we also use the analyzer to canonicalized queries (for view definition), // we do not eliminate subqueries or compute current time in the analyzer. Batch("Finish Analysis", Once, + RewritePredicateSubquery, EliminateSubqueryAliases, ComputeCurrentTime, GetCurrentDatabase(sessionCatalog), @@ -1446,3 +1448,114 @@ object EmbedSerializerInFilter extends Rule[LogicalPlan] { } } } + +/** + * This rule rewrites predicate sub-queries into left semi/anti joins. The following predicates + * are supported: + * a. EXISTS/NOT EXISTS will be rewritten as semi/anti join, unresolved conditions in Filter + * will be pulled out as the join conditions. + * b. IN/NOT IN will be rewritten as semi/anti join, unresolved conditions in the Filter will + * be pulled out as join conditions, value = selected column will also be used as join + * condition. + */ +object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { + /** + * Pull out all correlated predicates from a given sub-query. This method removes the correlated + * predicates from sub-query [[Filter]]s and adds the references of these predicates to + * all intermediate [[Project]] clauses (if they are missing) in order to be able to evaluate the + * predicates in the join condition. + * + * This method returns the rewritten sub-query and the combined (AND) extracted predicate. + */ + private def pullOutCorrelatedPredicates( + subquery: LogicalPlan, + query: LogicalPlan): (LogicalPlan, Seq[Expression]) = { + val references = query.outputSet + val predicateMap = mutable.Map.empty[LogicalPlan, Seq[Expression]] + val transformed = subquery transformUp { + case f @ Filter(cond, child) => + // Find all correlated predicates. + val (correlated, local) = splitConjunctivePredicates(cond).partition { e => + e.references.intersect(references).nonEmpty + } + // Rewrite the filter without the correlated predicates if any. + correlated match { + case Nil => f + case xs if local.nonEmpty => + val newFilter = Filter(local.reduce(And), child) + predicateMap += newFilter -> correlated + newFilter + case xs => + predicateMap += child -> correlated + child + } + case p @ Project(expressions, child) => + // Find all pulled out predicates defined in the Project's subtree. + val localPredicates = p.collect(predicateMap).flatten + + // Determine which correlated predicate references are missing from this project. + val localPredicateReferences = localPredicates + .map(_.references) + .reduceOption(_ ++ _) + .getOrElse(AttributeSet.empty) + val missingReferences = localPredicateReferences -- p.references -- query.outputSet + + // Create a new project if we need to add missing references. + if (missingReferences.nonEmpty) { + Project(expressions ++ missingReferences, child) + } else { + p + } + } + (transformed, predicateMap.values.flatten.toSeq) + } + + /** + * Prepare an [[InSubQuery]] by rewriting it (in case of correlated predicates) and by + * constructing the required join condition. Both the rewritten subquery and the constructed + * join condition are returned. + */ + private def pullOutCorrelatedPredicates( + in: InSubQuery, + query: LogicalPlan): (LogicalPlan, Seq[Expression]) = { + val (resolved, joinCondition) = pullOutCorrelatedPredicates(in.query, query) + val conditions = joinCondition ++ in.expressions.zip(resolved.output).map(EqualTo.tupled) + (resolved, conditions) + } + + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case f @ Filter(condition, child) => + val (withSubquery, withoutSubquery) = + splitConjunctivePredicates(condition).partition(PredicateSubquery.hasPredicateSubquery) + + // Construct the pruned filter condition. + val newFilter: LogicalPlan = withoutSubquery match { + case Nil => child + case conditions => Filter(conditions.reduce(And), child) + } + + // Filter the plan by applying left semi and left anti joins. + withSubquery.foldLeft(newFilter) { + case (p, Exists(sub)) => + val (resolved, conditions) = pullOutCorrelatedPredicates(sub, p) + Join(p, resolved, LeftSemi, conditions.reduceOption(And)) + case (p, Not(Exists(sub))) => + val (resolved, conditions) = pullOutCorrelatedPredicates(sub, p) + Join(p, resolved, LeftAnti, conditions.reduceOption(And)) + case (p, in: InSubQuery) => + val (resolved, conditions) = pullOutCorrelatedPredicates(in, p) + Join(p, resolved, LeftSemi, conditions.reduceOption(And)) + case (p, Not(in: InSubQuery)) => + val (resolved, conditions) = pullOutCorrelatedPredicates(in, p) + // This is a NULL-aware (left) anti join (NAAJ). + // Construct the condition. A NULL in one of the conditions is regarded as a positive + // result; such a row will be filtered out by the Anti-Join operator. + val anyNull = conditions.map(IsNull).reduceLeft(Or) + val condition = conditions.reduceLeft(And) + + // Note that will almost certainly be planned as a Broadcast Nested Loop join. Use EXISTS + // if performance matters to you. + Join(p, resolved, LeftAnti, Option(Or(anyNull, condition))) + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index aa59f3fb2a4a4..1c067621df524 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -391,9 +391,13 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { // Having val withHaving = withProject.optional(having) { - // Note that we added a cast to boolean. If the expression itself is already boolean, - // the optimizer will get rid of the unnecessary cast. - Filter(Cast(expression(having), BooleanType), withProject) + // Note that we add a cast to non-predicate expressions. If the expression itself is + // already boolean, the optimizer will get rid of the unnecessary cast. + val predicate = expression(having) match { + case p: Predicate => p + case e => Cast(e, BooleanType) + } + Filter(predicate, withProject) } // Distinct @@ -866,10 +870,10 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { } /** - * Create a filtering correlated sub-query. This is not supported yet. + * Create a filtering correlated sub-query (EXISTS). */ override def visitExists(ctx: ExistsContext): Expression = { - throw new ParseException("EXISTS clauses are not supported.", ctx) + Exists(plan(ctx.query)) } /** @@ -944,7 +948,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { GreaterThanOrEqual(e, expression(ctx.lower)), LessThanOrEqual(e, expression(ctx.upper)))) case SqlBaseParser.IN if ctx.query != null => - throw new ParseException("IN with a Sub-query is currently not supported.", ctx) + invertIfNotDefined(InSubQuery(e, plan(ctx.query))) case SqlBaseParser.IN => invertIfNotDefined(In(e, ctx.expression.asScala.map(expression))) case SqlBaseParser.LIKE => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index ad101d1c406b8..a90636d278673 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -24,8 +24,8 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete, Count} +import org.apache.spark.sql.catalyst.plans.{Inner, LeftOuter, RightOuter} import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, MapData} import org.apache.spark.sql.types._ @@ -444,4 +444,60 @@ class AnalysisErrorSuite extends AnalysisTest { assertAnalysisError(plan2, "map type expression `a` cannot be used in join conditions" :: Nil) } + + test("PredicateSubQuery is used outside of a filter") { + val a = AttributeReference("a", IntegerType)() + val b = AttributeReference("b", IntegerType)() + val plan = Project( + Seq(a, Alias(InSubQuery(a, LocalRelation(b)), "c")()), + LocalRelation(a)) + assertAnalysisError(plan, "Predicate sub-queries can only be used in a Filter" :: Nil) + } + + test("PredicateSubQuery is used is a nested condition") { + val a = AttributeReference("a", IntegerType)() + val b = AttributeReference("b", IntegerType)() + val c = AttributeReference("c", BooleanType)() + val plan1 = Filter(Cast(InSubQuery(a, LocalRelation(b)), BooleanType), LocalRelation(a)) + assertAnalysisError(plan1, "Predicate sub-queries cannot be used in nested conditions" :: Nil) + + val plan2 = Filter(Or(InSubQuery(a, LocalRelation(b)), c), LocalRelation(a, c)) + assertAnalysisError(plan2, "Predicate sub-queries cannot be used in nested conditions" :: Nil) + } + + test("PredicateSubQuery correlated predicate is nested in an illegal plan") { + val a = AttributeReference("a", IntegerType)() + val b = AttributeReference("b", IntegerType)() + val c = AttributeReference("c", IntegerType)() + + val plan1 = Filter( + Exists( + Join( + LocalRelation(b), + Filter(EqualTo(a, c), LocalRelation(c)), + LeftOuter, + Option(EqualTo(b, c)))), + LocalRelation(a)) + assertAnalysisError(plan1, "Accessing outer query column is not allowed in" :: Nil) + + val plan2 = Filter( + Exists( + Join( + Filter(EqualTo(a, c), LocalRelation(c)), + LocalRelation(b), + RightOuter, + Option(EqualTo(b, c)))), + LocalRelation(a)) + assertAnalysisError(plan2, "Accessing outer query column is not allowed in" :: Nil) + + val plan3 = Filter( + Exists(Aggregate(Seq.empty, Seq.empty, Filter(EqualTo(a, c), LocalRelation(c)))), + LocalRelation(a)) + assertAnalysisError(plan3, "Accessing outer query column is not allowed in" :: Nil) + + val plan4 = Filter( + Exists(Union(LocalRelation(b), Filter(EqualTo(a, c), LocalRelation(c)))), + LocalRelation(a)) + assertAnalysisError(plan4, "Accessing outer query column is not allowed in" :: Nil) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala index db96bfb652120..6da3eaea3d850 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala @@ -60,8 +60,8 @@ class ErrorParserSuite extends SparkFunSuite { intercept("select *\nfrom r\norder by q\ncluster by q", 3, 0, "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported", "^^^") - intercept("select * from r where a in (select * from t)", 1, 24, - "IN with a Sub-query is currently not supported", - "------------------------^^^") + intercept("select * from r except all select * from t", 1, 0, + "EXCEPT ALL is not supported", + "^^^") } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 6f40ec67ec6e0..d1dc8d621fb4d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -113,7 +113,9 @@ class ExpressionParserSuite extends PlanTest { } test("exists expression") { - intercept("exists (select 1 from b where b.x = a.x)", "EXISTS clauses are not supported") + assertEqual( + "exists (select 1 from b where b.x = a.x)", + Exists(table("b").where(Symbol("b.x") === Symbol("a.x")).select(1))) } test("comparison expressions") { @@ -139,7 +141,9 @@ class ExpressionParserSuite extends PlanTest { } test("in sub-query") { - intercept("a in (select b from c)", "IN with a Sub-query is currently not supported") + assertEqual( + "a in (select b from c)", + InSubQuery('a, table("c").select('b))) } test("like expressions") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 411e2372f2e07..a1ca55c262fba 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -107,7 +107,7 @@ class PlanParserSuite extends PlanTest { assertEqual("select a, b from db.c where x < 1", table("db", "c").where('x < 1).select('a, 'b)) assertEqual( "select a, b from db.c having x < 1", - table("db", "c").select('a, 'b).where(('x < 1).cast(BooleanType))) + table("db", "c").select('a, 'b).where('x < 1)) assertEqual("select distinct a, b from db.c", Distinct(table("db", "c").select('a, 'b))) assertEqual("select all a, b from db.c", table("db", "c").select('a, 'b)) } @@ -405,7 +405,7 @@ class PlanParserSuite extends PlanTest { "select g from t group by g having a > (select b from s)", table("t") .groupBy('g)('g) - .where(('a > ScalarSubquery(table("s").select('b))).cast(BooleanType))) + .where('a > ScalarSubquery(table("s").select('b)))) } test("table reference") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index b3e8b37a2ebfc..71b6a97852966 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -18,8 +18,9 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.catalyst.{expressions, InternalRow} -import org.apache.spark.sql.catalyst.expressions.{ExprId, Literal, SubqueryExpression} +import org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Expression, ExprId, Literal, SubqueryExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule @@ -42,6 +43,7 @@ case class ScalarSubquery( override def plan: SparkPlan = Subquery(simpleString, executedPlan) override def dataType: DataType = executedPlan.schema.fields.head.dataType + override def children: Seq[Expression] = Nil override def nullable: Boolean = true override def toString: String = s"subquery#${exprId.id}" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 2dca792c83bf0..cbacb5e1033f0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import java.util.{Locale, TimeZone} +import java.util.{ArrayDeque, Locale, TimeZone} import scala.collection.JavaConverters._ import scala.util.control.NonFatal @@ -35,6 +35,8 @@ import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.streaming.MemoryPlan import org.apache.spark.sql.types.ObjectType + + abstract class QueryTest extends PlanTest { protected def sqlContext: SQLContext @@ -47,6 +49,7 @@ abstract class QueryTest extends PlanTest { /** * Runs the plan and makes sure the answer contains all of the keywords, or the * none of keywords are listed in the answer + * * @param df the [[DataFrame]] to be executed * @param exists true for make sure the keywords are listed in the output, otherwise * to make sure none of the keyword are not listed in the output @@ -119,6 +122,7 @@ abstract class QueryTest extends PlanTest { /** * Runs the plan and makes sure the answer matches the expected result. + * * @param df the [[DataFrame]] to be executed * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. */ @@ -158,6 +162,7 @@ abstract class QueryTest extends PlanTest { /** * Runs the plan and makes sure the answer is within absTol of the expected result. + * * @param dataFrame the [[DataFrame]] to be executed * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. * @param absTol the absolute tolerance between actual and expected answers. @@ -198,7 +203,10 @@ abstract class QueryTest extends PlanTest { } private def checkJsonFormat(df: DataFrame): Unit = { + // Get the analyzed plan and rewrite the PredicateSubqueries in order to make sure that + // RDD and Data resolution does not break. val logicalPlan = df.queryExecution.analyzed + // bypass some cases that we can't handle currently. logicalPlan.transform { case _: ObjectConsumer => return @@ -236,9 +244,27 @@ abstract class QueryTest extends PlanTest { // RDDs/data are not serializable to JSON, so we need to collect LogicalPlans that contains // these non-serializable stuff, and use these original ones to replace the null-placeholders // in the logical plans parsed from JSON. - var logicalRDDs = logicalPlan.collect { case l: LogicalRDD => l } - var localRelations = logicalPlan.collect { case l: LocalRelation => l } - var inMemoryRelations = logicalPlan.collect { case i: InMemoryRelation => i } + val logicalRDDs = new ArrayDeque[LogicalRDD]() + val localRelations = new ArrayDeque[LocalRelation]() + val inMemoryRelations = new ArrayDeque[InMemoryRelation]() + def collectData: (LogicalPlan => Unit) = { + case l: LogicalRDD => + logicalRDDs.offer(l) + case l: LocalRelation => + localRelations.offer(l) + case i: InMemoryRelation => + inMemoryRelations.offer(i) + case p => + p.expressions.foreach { + _.foreach { + case s: SubqueryExpression => + s.query.foreach(collectData) + case _ => + } + } + } + logicalPlan.foreach(collectData) + val jsonBackPlan = try { TreeNode.fromJSON[LogicalPlan](jsonString, sqlContext.sparkContext) @@ -253,18 +279,15 @@ abstract class QueryTest extends PlanTest { """.stripMargin, e) } - val normalized2 = jsonBackPlan transformDown { + def renormalize: PartialFunction[LogicalPlan, LogicalPlan] = { case l: LogicalRDD => - val origin = logicalRDDs.head - logicalRDDs = logicalRDDs.drop(1) + val origin = logicalRDDs.pop() LogicalRDD(l.output, origin.rdd)(sqlContext) case l: LocalRelation => - val origin = localRelations.head - localRelations = localRelations.drop(1) + val origin = localRelations.pop() l.copy(data = origin.data) case l: InMemoryRelation => - val origin = inMemoryRelations.head - inMemoryRelations = inMemoryRelations.drop(1) + val origin = inMemoryRelations.pop() InMemoryRelation( l.output, l.useCompression, @@ -275,7 +298,13 @@ abstract class QueryTest extends PlanTest { origin.cachedColumnBuffers, l._statistics, origin._batchStats) + case p => + p.transformExpressions { + case s: SubqueryExpression => + s.withNewPlan(s.query.transformDown(renormalize)) + } } + val normalized2 = jsonBackPlan.transformDown(renormalize) assert(logicalRDDs.isEmpty) assert(localRelations.isEmpty) @@ -309,6 +338,7 @@ object QueryTest { * If there was exception during the execution or the contents of the DataFrame does not * match the expected result, an error message will be returned. Otherwise, a [[None]] will * be returned. + * * @param df the [[DataFrame]] to be executed * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. */ @@ -383,6 +413,7 @@ object QueryTest { /** * Runs the plan and makes sure the answer is within absTol of the expected result. + * * @param actualAnswer the actual result in a [[Row]]. * @param expectedAnswer the expected result in a[[Row]]. * @param absTol the absolute tolerance between actual and expected answers. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 21b19fe7df8b2..5742983fb9d07 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -22,6 +22,38 @@ import org.apache.spark.sql.test.SharedSQLContext class SubquerySuite extends QueryTest with SharedSQLContext { import testImplicits._ + setupTestData() + + val row = identity[(java.lang.Integer, java.lang.Double)](_) + + lazy val l = Seq( + row(1, 2.0), + row(1, 2.0), + row(2, 1.0), + row(2, 1.0), + row(3, 3.0), + row(null, null), + row(null, 5.0), + row(6, null)).toDF("a", "b") + + lazy val r = Seq( + row(2, 3.0), + row(2, 3.0), + row(3, 2.0), + row(4, 1.0), + row(null, null), + row(null, 5.0), + row(6, null)).toDF("c", "d") + + lazy val t = r.filter($"c".isNotNull && $"d".isNotNull) + + protected override def beforeAll(): Unit = { + super.beforeAll() + l.registerTempTable("l") + r.registerTempTable("r") + t.registerTempTable("t") + } + test("simple uncorrelated scalar subquery") { assertResult(Array(Row(1))) { sql("select (select 1 as b) as b").collect() @@ -80,4 +112,70 @@ class SubquerySuite extends QueryTest with SharedSQLContext { " where key = (select max(key) from subqueryData) - 1)").collect() } } + + test("EXISTS predicate subquery") { + checkAnswer( + sql("select * from l where exists(select * from r where l.a = r.c)"), + Row(2, 1.0) :: Row(2, 1.0) :: Row(3, 3.0) :: Row(6, null) :: Nil) + + checkAnswer( + sql("select * from l where exists(select * from r where l.a = r.c) and l.a <= 2"), + Row(2, 1.0) :: Row(2, 1.0) :: Nil) + } + + test("NOT EXISTS predicate subquery") { + checkAnswer( + sql("select * from l where not exists(select * from r where l.a = r.c)"), + Row(1, 2.0) :: Row(1, 2.0) :: Row(null, null) :: Row(null, 5.0) :: Nil) + + checkAnswer( + sql("select * from l where not exists(select * from r where l.a = r.c and l.b < r.d)"), + Row(1, 2.0) :: Row(1, 2.0) :: Row(3, 3.0) :: + Row(null, null) :: Row(null, 5.0) :: Row(6, null) :: Nil) + } + + test("IN predicate subquery") { + checkAnswer( + sql("select * from l where l.a in (select c from r)"), + Row(2, 1.0) :: Row(2, 1.0) :: Row(3, 3.0) :: Row(6, null) :: Nil) + + checkAnswer( + sql("select * from l where l.a in (select c from r where l.b < r.d)"), + Row(2, 1.0) :: Row(2, 1.0) :: Nil) + + checkAnswer( + sql("select * from l where l.a in (select c from r) and l.a > 2 and l.b is not null"), + Row(3, 3.0) :: Nil) + } + + test("NOT IN predicate subquery") { + checkAnswer( + sql("select * from l where a not in(select c from r)"), + Nil) + + checkAnswer( + sql("select * from l where a not in(select c from r where c is not null)"), + Row(1, 2.0) :: Row(1, 2.0) :: Nil) + + checkAnswer( + sql("select * from l where a not in(select c from t where b < d)"), + Row(1, 2.0) :: Row(1, 2.0) :: Row(3, 3.0) :: Nil) + + // Empty sub-query + checkAnswer( + sql("select * from l where a not in(select c from r where c > 10 and b < d)"), + Row(1, 2.0) :: Row(1, 2.0) :: Row(2, 1.0) :: Row(2, 1.0) :: + Row(3, 3.0) :: Row(null, null) :: Row(null, 5.0) :: Row(6, null) :: Nil) + + } + + test("complex IN predicate subquery") { + checkAnswer( + sql("select * from l where (a, b) not in(select c, d from r)"), + Nil) + + checkAnswer( + sql("select * from l where (a, b) not in(select c, d from t) and (a + b) is not null"), + Row(1, 2.0) :: Row(1, 2.0) :: Row(2, 1.0) :: Row(2, 1.0) :: Row(3, 3.0) :: Nil) + } } From 008a8bbef0d3475610c13fff778a425900912650 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Tue, 19 Apr 2016 15:55:21 -0700 Subject: [PATCH 37/69] [SPARK-14733] Allow custom timing control in microbenchmarks ## What changes were proposed in this pull request? The current benchmark framework runs a code block for several iterations and reports statistics. However there is no way to exclude per-iteration setup time from the overall results. This PR adds a timer control object passed into the closure that can be used for this purpose. ## How was this patch tested? Existing benchmark code. Also see https://github.com/apache/spark/pull/12490 Author: Eric Liang Closes #12502 from ericl/spark-14733. --- .../org/apache/spark/util/Benchmark.scala | 57 ++++++++++++++++--- 1 file changed, 49 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Benchmark.scala b/core/src/main/scala/org/apache/spark/util/Benchmark.scala index 3718542810d0b..1fc0ad7a4d6d3 100644 --- a/core/src/main/scala/org/apache/spark/util/Benchmark.scala +++ b/core/src/main/scala/org/apache/spark/util/Benchmark.scala @@ -42,7 +42,24 @@ private[spark] class Benchmark( outputPerIteration: Boolean = false) { val benchmarks = mutable.ArrayBuffer.empty[Benchmark.Case] + /** + * Adds a case to run when run() is called. The given function will be run for several + * iterations to collect timing statistics. + */ def addCase(name: String)(f: Int => Unit): Unit = { + addTimerCase(name) { timer => + timer.startTiming() + f(timer.iteration) + timer.stopTiming() + } + } + + /** + * Adds a case with manual timing control. When the function is run, timing does not start + * until timer.startTiming() is called within the given function. The corresponding + * timer.stopTiming() method must be called before the function returns. + */ + def addTimerCase(name: String)(f: Benchmark.Timer => Unit): Unit = { benchmarks += Benchmark.Case(name, f) } @@ -84,7 +101,34 @@ private[spark] class Benchmark( } private[spark] object Benchmark { - case class Case(name: String, fn: Int => Unit) + + /** + * Object available to benchmark code to control timing e.g. to exclude set-up time. + * + * @param iteration specifies this is the nth iteration of running the benchmark case + */ + class Timer(val iteration: Int) { + private var accumulatedTime: Long = 0L + private var timeStart: Long = 0L + + def startTiming(): Unit = { + assert(timeStart == 0L, "Already started timing.") + timeStart = System.nanoTime + } + + def stopTiming(): Unit = { + assert(timeStart != 0L, "Have not started timing.") + accumulatedTime += System.nanoTime - timeStart + timeStart = 0L + } + + def totalTime(): Long = { + assert(timeStart == 0L, "Have not stopped timing.") + accumulatedTime + } + } + + case class Case(name: String, fn: Timer => Unit) case class Result(avgMs: Double, bestRate: Double, bestMs: Double) /** @@ -123,15 +167,12 @@ private[spark] object Benchmark { * Runs a single function `f` for iters, returning the average time the function took and * the rate of the function. */ - def measure(num: Long, iters: Int, outputPerIteration: Boolean)(f: Int => Unit): Result = { + def measure(num: Long, iters: Int, outputPerIteration: Boolean)(f: Timer => Unit): Result = { val runTimes = ArrayBuffer[Long]() for (i <- 0 until iters + 1) { - val start = System.nanoTime() - - f(i) - - val end = System.nanoTime() - val runTime = end - start + val timer = new Benchmark.Timer(i) + f(timer) + val runTime = timer.totalTime() if (i > 0) { runTimes += runTime } From ecd877e8335ff6bb06c96d3045ccade80676e714 Mon Sep 17 00:00:00 2001 From: felixcheung Date: Tue, 19 Apr 2016 15:59:47 -0700 Subject: [PATCH 38/69] [SPARK-12224][SPARKR] R support for JDBC source Add R API for `read.jdbc`, `write.jdbc`. Tested this quite a bit manually with different combinations of parameters. It's not clear if we could have automated tests in R for this - Scala `JDBCSuite` depends on Java H2 in-memory database. Refactored some code into util so they could be tested. Core's R SerDe code needs to be updated to allow access to java.util.Properties as `jobj` handle which is required by DataFrameReader/Writer's `jdbc` method. It would be possible, though more code to add a `sql/r/SQLUtils` helper function. Tested: ``` # with postgresql ../bin/sparkR --driver-class-path /usr/share/java/postgresql-9.4.1207.jre7.jar # read.jdbc df <- read.jdbc(sqlContext, "jdbc:postgresql://localhost/db", "films2", user = "user", password = "12345") df <- read.jdbc(sqlContext, "jdbc:postgresql://localhost/db", "films2", user = "user", password = 12345) # partitionColumn and numPartitions test df <- read.jdbc(sqlContext, "jdbc:postgresql://localhost/db", "films2", partitionColumn = "did", lowerBound = 0, upperBound = 200, numPartitions = 4, user = "user", password = 12345) a <- SparkR:::toRDD(df) SparkR:::getNumPartitions(a) [1] 4 SparkR:::collectPartition(a, 2L) # defaultParallelism test df <- read.jdbc(sqlContext, "jdbc:postgresql://localhost/db", "films2", partitionColumn = "did", lowerBound = 0, upperBound = 200, user = "user", password = 12345) SparkR:::getNumPartitions(a) [1] 2 # predicates test df <- read.jdbc(sqlContext, "jdbc:postgresql://localhost/db", "films2", predicates = list("did<=105"), user = "user", password = 12345) count(df) == 1 # write.jdbc, default save mode "error" irisDf <- as.DataFrame(sqlContext, iris) write.jdbc(irisDf, "jdbc:postgresql://localhost/db", "films2", user = "user", password = "12345") "error, already exists" write.jdbc(irisDf, "jdbc:postgresql://localhost/db", "iris", user = "user", password = "12345") ``` Author: felixcheung Closes #10480 from felixcheung/rreadjdbc. --- R/pkg/NAMESPACE | 2 + R/pkg/R/DataFrame.R | 39 ++++++++++++- R/pkg/R/SQLContext.R | 58 +++++++++++++++++++ R/pkg/R/generics.R | 6 ++ R/pkg/R/utils.R | 11 ++++ R/pkg/inst/tests/testthat/test_utils.R | 24 ++++++++ .../scala/org/apache/spark/api/r/SerDe.scala | 7 +++ 7 files changed, 146 insertions(+), 1 deletion(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 94ac7e7df7147..10b9d16279308 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -101,6 +101,7 @@ exportMethods("arrange", "withColumn", "withColumnRenamed", "write.df", + "write.jdbc", "write.json", "write.parquet", "write.text") @@ -284,6 +285,7 @@ export("as.DataFrame", "loadDF", "parquetFile", "read.df", + "read.jdbc", "read.json", "read.parquet", "read.text", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index a64a013b654ef..ddb056fa71e7e 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -2363,7 +2363,7 @@ setMethod("with", #' @examples \dontrun{ #' # Create a DataFrame from the Iris dataset #' irisDF <- createDataFrame(sqlContext, iris) -#' +#' #' # Show the structure of the DataFrame #' str(irisDF) #' } @@ -2468,3 +2468,40 @@ setMethod("drop", function(x) { base::drop(x) }) + +#' Saves the content of the DataFrame to an external database table via JDBC +#' +#' Additional JDBC database connection properties can be set (...) +#' +#' Also, mode is used to specify the behavior of the save operation when +#' data already exists in the data source. There are four modes: \cr +#' append: Contents of this DataFrame are expected to be appended to existing data. \cr +#' overwrite: Existing data is expected to be overwritten by the contents of this DataFrame. \cr +#' error: An exception is expected to be thrown. \cr +#' ignore: The save operation is expected to not save the contents of the DataFrame +#' and to not change the existing data. \cr +#' +#' @param x A SparkSQL DataFrame +#' @param url JDBC database url of the form `jdbc:subprotocol:subname` +#' @param tableName The name of the table in the external database +#' @param mode One of 'append', 'overwrite', 'error', 'ignore' save mode (it is 'error' by default) +#' @family DataFrame functions +#' @rdname write.jdbc +#' @name write.jdbc +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlContext <- sparkRSQL.init(sc) +#' jdbcUrl <- "jdbc:mysql://localhost:3306/databasename" +#' write.jdbc(df, jdbcUrl, "table", user = "username", password = "password") +#' } +setMethod("write.jdbc", + signature(x = "DataFrame", url = "character", tableName = "character"), + function(x, url, tableName, mode = "error", ...){ + jmode <- convertToJSaveMode(mode) + jprops <- varargsToJProperties(...) + write <- callJMethod(x@sdf, "write") + write <- callJMethod(write, "mode", jmode) + invisible(callJMethod(write, "jdbc", url, tableName, jprops)) + }) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 16a2578678cd3..b726c1e1b9f2c 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -583,3 +583,61 @@ createExternalTable <- function(sqlContext, tableName, path = NULL, source = NUL sdf <- callJMethod(sqlContext, "createExternalTable", tableName, source, options) dataFrame(sdf) } + +#' Create a DataFrame representing the database table accessible via JDBC URL +#' +#' Additional JDBC database connection properties can be set (...) +#' +#' Only one of partitionColumn or predicates should be set. Partitions of the table will be +#' retrieved in parallel based on the `numPartitions` or by the predicates. +#' +#' Don't create too many partitions in parallel on a large cluster; otherwise Spark might crash +#' your external database systems. +#' +#' @param sqlContext SQLContext to use +#' @param url JDBC database url of the form `jdbc:subprotocol:subname` +#' @param tableName the name of the table in the external database +#' @param partitionColumn the name of a column of integral type that will be used for partitioning +#' @param lowerBound the minimum value of `partitionColumn` used to decide partition stride +#' @param upperBound the maximum value of `partitionColumn` used to decide partition stride +#' @param numPartitions the number of partitions, This, along with `lowerBound` (inclusive), +#' `upperBound` (exclusive), form partition strides for generated WHERE +#' clause expressions used to split the column `partitionColumn` evenly. +#' This defaults to SparkContext.defaultParallelism when unset. +#' @param predicates a list of conditions in the where clause; each one defines one partition +#' @return DataFrame +#' @rdname read.jdbc +#' @name read.jdbc +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlContext <- sparkRSQL.init(sc) +#' jdbcUrl <- "jdbc:mysql://localhost:3306/databasename" +#' df <- read.jdbc(sqlContext, jdbcUrl, "table", predicates = list("field<=123"), user = "username") +#' df2 <- read.jdbc(sqlContext, jdbcUrl, "table2", partitionColumn = "index", lowerBound = 0, +#' upperBound = 10000, user = "username", password = "password") +#' } + +read.jdbc <- function(sqlContext, url, tableName, + partitionColumn = NULL, lowerBound = NULL, upperBound = NULL, + numPartitions = 0L, predicates = list(), ...) { + jprops <- varargsToJProperties(...) + + read <- callJMethod(sqlContext, "read") + if (!is.null(partitionColumn)) { + if (is.null(numPartitions) || numPartitions == 0) { + sc <- callJMethod(sqlContext, "sparkContext") + numPartitions <- callJMethod(sc, "defaultParallelism") + } else { + numPartitions <- numToInt(numPartitions) + } + sdf <- callJMethod(read, "jdbc", url, tableName, as.character(partitionColumn), + numToInt(lowerBound), numToInt(upperBound), numPartitions, jprops) + } else if (length(predicates) > 0) { + sdf <- callJMethod(read, "jdbc", url, tableName, as.list(as.character(predicates)), jprops) + } else { + sdf <- callJMethod(read, "jdbc", url, tableName, jprops) + } + dataFrame(sdf) +} diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index ecdeea5ec4912..4ef05d56bfb4f 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -577,6 +577,12 @@ setGeneric("saveDF", function(df, path, source = NULL, mode = "error", ...) { standardGeneric("saveDF") }) +#' @rdname write.jdbc +#' @export +setGeneric("write.jdbc", function(x, url, tableName, mode = "error", ...) { + standardGeneric("write.jdbc") +}) + #' @rdname write.json #' @export setGeneric("write.json", function(x, path) { standardGeneric("write.json") }) diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index fb6575cb42907..b425ccf6e7a36 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -650,3 +650,14 @@ convertToJSaveMode <- function(mode) { jmode <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "saveMode", mode) jmode } + +varargsToJProperties <- function(...) { + pairs <- list(...) + props <- newJObject("java.util.Properties") + if (length(pairs) > 0) { + lapply(ls(pairs), function(k) { + callJMethod(props, "setProperty", as.character(k), as.character(pairs[[k]])) + }) + } + props +} diff --git a/R/pkg/inst/tests/testthat/test_utils.R b/R/pkg/inst/tests/testthat/test_utils.R index 4218138f641d1..01694ab5c4f61 100644 --- a/R/pkg/inst/tests/testthat/test_utils.R +++ b/R/pkg/inst/tests/testthat/test_utils.R @@ -140,3 +140,27 @@ test_that("cleanClosure on R functions", { expect_equal(ls(env), "aBroadcast") expect_equal(get("aBroadcast", envir = env, inherits = FALSE), aBroadcast) }) + +test_that("varargsToJProperties", { + jprops <- newJObject("java.util.Properties") + expect_true(class(jprops) == "jobj") + + jprops <- varargsToJProperties(abc = "123") + expect_true(class(jprops) == "jobj") + expect_equal(callJMethod(jprops, "getProperty", "abc"), "123") + + jprops <- varargsToJProperties(abc = "abc", b = 1) + expect_equal(callJMethod(jprops, "getProperty", "abc"), "abc") + expect_equal(callJMethod(jprops, "getProperty", "b"), "1") + + jprops <- varargsToJProperties() + expect_equal(callJMethod(jprops, "size"), 0L) +}) + +test_that("convertToJSaveMode", { + s <- convertToJSaveMode("error") + expect_true(class(s) == "jobj") + expect_match(capture.output(print.jobj(s)), "Java ref type org.apache.spark.sql.SaveMode id ") + expect_error(convertToJSaveMode("foo"), + 'mode should be one of "append", "overwrite", "error", "ignore"') #nolint +}) diff --git a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala index 48df5bedd6e41..8e4e80a24acee 100644 --- a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala +++ b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala @@ -356,6 +356,13 @@ private[spark] object SerDe { writeInt(dos, v.length) v.foreach(elem => writeObject(dos, elem)) + // Handle Properties + // This must be above the case java.util.Map below. + // (Properties implements Map and will be serialized as map otherwise) + case v: java.util.Properties => + writeType(dos, "jobj") + writeJObj(dos, value) + // Handle map case v: java.util.Map[_, _] => writeType(dos, "map") From a685e65a4ca0b300b12103fccbda29cb08221f5d Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 19 Apr 2016 16:15:06 -0700 Subject: [PATCH 39/69] Revert "[SPARK-14719] WriteAheadLogBasedBlockHandler should ignore BlockManager put errors" This reverts commit ed2de0299a5a54b566b91ae9f47b6626c484c1d3. --- .../receiver/ReceivedBlockHandler.scala | 22 +++------ .../streaming/ReceivedBlockHandlerSuite.scala | 49 +++++++++---------- 2 files changed, 31 insertions(+), 40 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala index a7d870500fd02..c4bc5cf3f6a58 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala @@ -20,7 +20,6 @@ package org.apache.spark.streaming.receiver import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration._ import scala.language.{existentials, postfixOps} -import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -190,19 +189,14 @@ private[streaming] class WriteAheadLogBasedBlockHandler( // Store the block in block manager val storeInBlockManagerFuture = Future { - try { - val putSucceeded = blockManager.putBytes( - blockId, - serializedBlock, - effectiveStorageLevel, - tellMaster = true) - if (!putSucceeded) { - logWarning( - s"Could not store $blockId to block manager with storage level $storageLevel") - } - } catch { - case NonFatal(t) => - logError(s"Could not store $blockId to block manager with storage level $storageLevel", t) + val putSucceeded = blockManager.putBytes( + blockId, + serializedBlock, + effectiveStorageLevel, + tellMaster = true) + if (!putSucceeded) { + throw new SparkException( + s"Could not store $blockId to block manager with storage level $storageLevel") } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index ea87b0d59fd9c..4be4882938df5 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -127,17 +127,7 @@ class ReceivedBlockHandlerSuite test("BlockManagerBasedBlockHandler - handle errors in storing block") { withBlockManagerBasedBlockHandler { handler => - // Handle error in iterator (e.g. divide-by-zero error) - intercept[Exception] { - val iterator = (10 to (-10, -1)).toIterator.map { _ / 0 } - handler.storeBlock(StreamBlockId(1, 1), IteratorBlock(iterator)) - } - - // Handler error in block manager storing (e.g. too big block) - intercept[SparkException] { - val byteBuffer = ByteBuffer.wrap(new Array[Byte](blockManagerSize + 1)) - handler.storeBlock(StreamBlockId(1, 1), ByteBufferBlock(byteBuffer)) - } + testErrorHandling(handler) } } @@ -177,15 +167,7 @@ class ReceivedBlockHandlerSuite test("WriteAheadLogBasedBlockHandler - handle errors in storing block") { withWriteAheadLogBasedBlockHandler { handler => - // Handle error in iterator (e.g. divide-by-zero error) - intercept[Exception] { - val iterator = (10 to (-10, -1)).toIterator.map { _ / 0 } - handler.storeBlock(StreamBlockId(1, 1), IteratorBlock(iterator)) - } - - // Throws no errors when storing blocks that are too large to be cached - val byteBuffer = ByteBuffer.wrap(new Array[Byte](blockManagerSize + 1)) - handler.storeBlock(StreamBlockId(1, 1), ByteBufferBlock(byteBuffer)) + testErrorHandling(handler) } } @@ -222,26 +204,26 @@ class ReceivedBlockHandlerSuite sparkConf.set("spark.storage.unrollFraction", "0.4") // Block Manager with 12000 * 0.4 = 4800 bytes of free space for unroll blockManager = createBlockManager(12000, sparkConf) - // This block is way too large to possibly be cached in memory: - def hugeBlock: IteratorBlock = IteratorBlock(List.fill(100)(new Array[Byte](1000)).iterator) // there is not enough space to store this block in MEMORY, // But BlockManager will be able to serialize this block to WAL // and hence count returns correct value. - testRecordcount(false, StorageLevel.MEMORY_ONLY, hugeBlock, blockManager, Some(100)) + testRecordcount(false, StorageLevel.MEMORY_ONLY, + IteratorBlock((List.fill(70)(new Array[Byte](100))).iterator), blockManager, Some(70)) // there is not enough space to store this block in MEMORY, // But BlockManager will be able to serialize this block to DISK // and hence count returns correct value. - testRecordcount(true, StorageLevel.MEMORY_AND_DISK, hugeBlock, blockManager, Some(100)) + testRecordcount(true, StorageLevel.MEMORY_AND_DISK, + IteratorBlock((List.fill(70)(new Array[Byte](100))).iterator), blockManager, Some(70)) // there is not enough space to store this block With MEMORY_ONLY StorageLevel. // BlockManager will not be able to unroll this block // and hence it will not tryToPut this block, resulting the SparkException storageLevel = StorageLevel.MEMORY_ONLY withBlockManagerBasedBlockHandler { handler => - intercept[SparkException] { - storeSingleBlock(handler, hugeBlock) + val thrown = intercept[SparkException] { + storeSingleBlock(handler, IteratorBlock((List.fill(70)(new Array[Byte](100))).iterator)) } } } @@ -364,6 +346,21 @@ class ReceivedBlockHandlerSuite storeAndVerify(blocks.map { b => ByteBufferBlock(dataToByteBuffer(b).toByteBuffer) }) } + /** Test error handling when blocks that cannot be stored */ + private def testErrorHandling(receivedBlockHandler: ReceivedBlockHandler) { + // Handle error in iterator (e.g. divide-by-zero error) + intercept[Exception] { + val iterator = (10 to (-10, -1)).toIterator.map { _ / 0 } + receivedBlockHandler.storeBlock(StreamBlockId(1, 1), IteratorBlock(iterator)) + } + + // Handler error in block manager storing (e.g. too big block) + intercept[SparkException] { + val byteBuffer = ByteBuffer.wrap(new Array[Byte](blockManagerSize + 1)) + receivedBlockHandler.storeBlock(StreamBlockId(1, 1), ByteBufferBlock(byteBuffer)) + } + } + /** Instantiate a BlockManagerBasedBlockHandler and run a code with it */ private def withBlockManagerBasedBlockHandler(body: BlockManagerBasedBlockHandler => Unit) { body(new BlockManagerBasedBlockHandler(blockManager, storageLevel)) From 3664142350afb6bf40a8bcb3508b56670603dae4 Mon Sep 17 00:00:00 2001 From: felixcheung Date: Tue, 19 Apr 2016 17:29:28 -0700 Subject: [PATCH 40/69] [SPARK-14717] [PYTHON] Scala, Python APIs for Dataset.unpersist differ in default blocking value ## What changes were proposed in this pull request? Change unpersist blocking parameter default value to match Scala ## How was this patch tested? unit tests, manual tests jkbradley davies Author: felixcheung Closes #12507 from felixcheung/pyunpersist. --- python/pyspark/sql/dataframe.py | 4 +++- python/pyspark/sql/tests.py | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index b4fa8368936a4..328bda6601565 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -326,9 +326,11 @@ def persist(self, storageLevel=StorageLevel.MEMORY_ONLY): return self @since(1.3) - def unpersist(self, blocking=True): + def unpersist(self, blocking=False): """Marks the :class:`DataFrame` as non-persistent, and remove all blocks for it from memory and disk. + + .. note:: `blocking` default has changed to False to match Scala in 2.0. """ self.is_cached = False self._jdf.unpersist(blocking) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index e4f79c911c0d9..d4c221d7125ca 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -362,7 +362,7 @@ def test_basic_functions(self): # cache and checkpoint self.assertFalse(df.is_cached) df.persist() - df.unpersist() + df.unpersist(True) df.cache() self.assertTrue(df.is_cached) self.assertEqual(2, df.count()) From 10f273d8db999cdc2e6c73bdbe98757de5d11676 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 19 Apr 2016 17:32:23 -0700 Subject: [PATCH 41/69] [SPARK-14407][SQL] Hides HadoopFsRelation related data source API into execution/datasources package #12178 ## What changes were proposed in this pull request? This PR moves `HadoopFsRelation` related data source API into `execution/datasources` package. Note that to avoid conflicts, this PR is based on #12153. Effective changes for this PR only consist of the last three commits. Will rebase after merging #12153. ## How was this patch tested? Existing tests. Author: Yin Huai Author: Cheng Lian Closes #12361 from liancheng/spark-14407-hide-hadoop-fs-relation. --- .../ml/source/libsvm/LibSVMRelation.scala | 2 +- project/MimaExcludes.scala | 4 + .../apache/spark/sql/DataFrameWriter.scala | 3 +- .../spark/sql/execution/ExistingRDD.scala | 5 +- .../execution/datasources/FileScanRDD.scala | 8 +- .../datasources/WriterContainer.scala | 1 - .../datasources/csv/CSVRelation.scala | 3 +- .../datasources/csv/DefaultSource.scala | 6 +- .../datasources/fileSourceInterfaces.scala | 530 ++++++++++++++++++ .../sql/execution/datasources/rules.scala | 2 +- .../datasources/text/DefaultSource.scala | 2 +- .../execution/streaming/FileStreamSink.scala | 2 +- .../streaming/StreamFileCatalog.scala | 3 +- .../apache/spark/sql/sources/interfaces.scala | 512 +---------------- .../parquet/ParquetFilterSuite.scala | 3 +- .../ParquetPartitionDiscoverySuite.scala | 3 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 3 +- .../spark/sql/hive/execution/commands.scala | 2 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 3 +- .../sql/hive/execution/SQLQuerySuite.scala | 3 +- .../spark/sql/hive/orc/OrcFilterSuite.scala | 3 +- .../apache/spark/sql/hive/parquetSuites.scala | 3 +- .../sql/sources/CommitFailureTestSource.scala | 5 +- .../sql/sources/SimpleTextRelation.scala | 2 +- .../sql/sources/hadoopFsRelationSuites.scala | 2 +- 25 files changed, 566 insertions(+), 549 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala index 2f1f2523fd11e..e8b0dd61f34b1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{AttributeReference, JoinedRow} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection -import org.apache.spark.sql.execution.datasources.{CaseInsensitiveMap, HadoopFileLinesReader, PartitionedFile} +import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ import org.apache.spark.util.SerializableConfiguration diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index b2c80afb533c1..7b15f5855886d 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -652,6 +652,10 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.shuffleWriteMetrics"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.shuffleReadMetrics"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.status.api.v1.TaskMetricDistributions.this") + ) ++ Seq( + // [SPARK-14407] Hides HadoopFsRelation related data source API into execution package + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.OutputWriter"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.OutputWriterFactory") ) case v if v.startsWith("1.6") => Seq( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 54d250867fbb3..1deeb8a2d25c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -27,11 +27,10 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, Project} -import org.apache.spark.sql.execution.datasources.{BucketSpec, CreateTableUsingAsSelect, DataSource} +import org.apache.spark.sql.execution.datasources.{BucketSpec, CreateTableUsingAsSelect, DataSource, HadoopFsRelation} import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils import org.apache.spark.sql.execution.streaming.{MemoryPlan, MemorySink, StreamExecution} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.HadoopFsRelation import org.apache.spark.util.Utils /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 1dc1b51e94519..12d03a7df8c53 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -26,11 +26,12 @@ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCo import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning} import org.apache.spark.sql.catalyst.util.toCommentSafeString +import org.apache.spark.sql.execution.datasources.HadoopFsRelation import org.apache.spark.sql.execution.datasources.parquet.{DefaultSource => ParquetSource} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.{BaseRelation, HadoopFsRelation} -import org.apache.spark.sql.types.{AtomicType, DataType} +import org.apache.spark.sql.sources.BaseRelation +import org.apache.spark.sql.types.DataType object RDDConversions { def productToRowRdd[A <: Product](data: RDD[A], outputTypes: Seq[DataType]): RDD[InternalRow] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index f86911e002a44..90694d9af4e01 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources -import org.apache.spark.{Partition, TaskContext} +import org.apache.spark.{Partition => RDDPartition, TaskContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.{InputFileNameHolder, RDD} import org.apache.spark.sql.SQLContext @@ -45,7 +45,7 @@ case class PartitionedFile( * * TODO: This currently does not take locality information about the files into account. */ -case class FilePartition(index: Int, files: Seq[PartitionedFile]) extends Partition +case class FilePartition(index: Int, files: Seq[PartitionedFile]) extends RDDPartition class FileScanRDD( @transient val sqlContext: SQLContext, @@ -53,7 +53,7 @@ class FileScanRDD( @transient val filePartitions: Seq[FilePartition]) extends RDD[InternalRow](sqlContext.sparkContext, Nil) { - override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { + override def compute(split: RDDPartition, context: TaskContext): Iterator[InternalRow] = { val iterator = new Iterator[Object] with AutoCloseable { private val inputMetrics = context.taskMetrics().inputMetrics private val existingBytesRead = inputMetrics.bytesRead @@ -130,5 +130,5 @@ class FileScanRDD( iterator.asInstanceOf[Iterator[InternalRow]] // This is an erasure hack. } - override protected def getPartitions: Array[Partition] = filePartitions.toArray + override protected def getPartitions: Array[RDDPartition] = filePartitions.toArray } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala index 815d1d01ef343..b9527db6d0092 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala @@ -33,7 +33,6 @@ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.UnsafeKVExternalSorter import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.{OutputWriter, OutputWriterFactory} import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import org.apache.spark.util.{SerializableConfiguration, Utils} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala index 54fb03b6d3bf7..ed40cd0c812ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala @@ -30,8 +30,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.execution.datasources.PartitionedFile -import org.apache.spark.sql.sources._ +import org.apache.spark.sql.execution.datasources.{OutputWriter, OutputWriterFactory, PartitionedFile} import org.apache.spark.sql.types._ object CSVRelation extends Logging { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/DefaultSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/DefaultSource.scala index 06a371b88bc02..34db10f822554 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/DefaultSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/DefaultSource.scala @@ -25,17 +25,15 @@ import org.apache.hadoop.io.{LongWritable, Text} import org.apache.hadoop.mapred.TextInputFormat import org.apache.hadoop.mapreduce._ -import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.JoinedRow import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection -import org.apache.spark.sql.execution.datasources.{CompressionCodecs, HadoopFileLinesReader, PartitionedFile} +import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{StringType, StructField, StructType} import org.apache.spark.util.SerializableConfiguration -import org.apache.spark.util.collection.BitSet /** * Provides access to CSV data from pure SQL statements. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala new file mode 100644 index 0000000000000..d37a939b544aa --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -0,0 +1,530 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import scala.collection.mutable +import scala.util.Try + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.apache.hadoop.mapred.{FileInputFormat, JobConf} +import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} + +import org.apache.spark.SparkContext +import org.apache.spark.annotation.Experimental +import org.apache.spark.internal.Logging +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.{expressions, CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.FileRelation +import org.apache.spark.sql.sources.{BaseRelation, Filter} +import org.apache.spark.sql.types.{StringType, StructType} +import org.apache.spark.util.SerializableConfiguration + +/** + * ::Experimental:: + * A factory that produces [[OutputWriter]]s. A new [[OutputWriterFactory]] is created on driver + * side for each write job issued when writing to a [[HadoopFsRelation]], and then gets serialized + * to executor side to create actual [[OutputWriter]]s on the fly. + * + * @since 1.4.0 + */ +@Experimental +abstract class OutputWriterFactory extends Serializable { + /** + * When writing to a [[HadoopFsRelation]], this method gets called by each task on executor side + * to instantiate new [[OutputWriter]]s. + * + * @param path Path of the file to which this [[OutputWriter]] is supposed to write. Note that + * this may not point to the final output file. For example, `FileOutputFormat` writes to + * temporary directories and then merge written files back to the final destination. In + * this case, `path` points to a temporary output file under the temporary directory. + * @param dataSchema Schema of the rows to be written. Partition columns are not included in the + * schema if the relation being written is partitioned. + * @param context The Hadoop MapReduce task context. + * @since 1.4.0 + */ + private[sql] def newInstance( + path: String, + bucketId: Option[Int], // TODO: This doesn't belong here... + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter +} + +/** + * ::Experimental:: + * [[OutputWriter]] is used together with [[HadoopFsRelation]] for persisting rows to the + * underlying file system. Subclasses of [[OutputWriter]] must provide a zero-argument constructor. + * An [[OutputWriter]] instance is created and initialized when a new output file is opened on + * executor side. This instance is used to persist rows to this single output file. + * + * @since 1.4.0 + */ +@Experimental +abstract class OutputWriter { + /** + * Persists a single row. Invoked on the executor side. When writing to dynamically partitioned + * tables, dynamic partition columns are not included in rows to be written. + * + * @since 1.4.0 + */ + def write(row: Row): Unit + + /** + * Closes the [[OutputWriter]]. Invoked on the executor side after all rows are persisted, before + * the task output is committed. + * + * @since 1.4.0 + */ + def close(): Unit + + private var converter: InternalRow => Row = _ + + protected[sql] def initConverter(dataSchema: StructType) = { + converter = + CatalystTypeConverters.createToScalaConverter(dataSchema).asInstanceOf[InternalRow => Row] + } + + protected[sql] def writeInternal(row: InternalRow): Unit = { + write(converter(row)) + } +} + +/** + * Acts as a container for all of the metadata required to read from a datasource. All discovery, + * resolution and merging logic for schemas and partitions has been removed. + * + * @param location A [[FileCatalog]] that can enumerate the locations of all the files that comprise + * this relation. + * @param partitionSchema The schema of the columns (if any) that are used to partition the relation + * @param dataSchema The schema of any remaining columns. Note that if any partition columns are + * present in the actual data files as well, they are preserved. + * @param bucketSpec Describes the bucketing (hash-partitioning of the files by some column values). + * @param fileFormat A file format that can be used to read and write the data in files. + * @param options Configuration used when reading / writing data. + */ +case class HadoopFsRelation( + sqlContext: SQLContext, + location: FileCatalog, + partitionSchema: StructType, + dataSchema: StructType, + bucketSpec: Option[BucketSpec], + fileFormat: FileFormat, + options: Map[String, String]) extends BaseRelation with FileRelation { + + val schema: StructType = { + val dataSchemaColumnNames = dataSchema.map(_.name.toLowerCase).toSet + StructType(dataSchema ++ partitionSchema.filterNot { column => + dataSchemaColumnNames.contains(column.name.toLowerCase) + }) + } + + def partitionSchemaOption: Option[StructType] = + if (partitionSchema.isEmpty) None else Some(partitionSchema) + def partitionSpec: PartitionSpec = location.partitionSpec() + + def refresh(): Unit = location.refresh() + + override def toString: String = + s"HadoopFiles" + + /** Returns the list of files that will be read when scanning this relation. */ + override def inputFiles: Array[String] = + location.allFiles().map(_.getPath.toUri.toString).toArray + + override def sizeInBytes: Long = location.allFiles().map(_.getLen).sum +} + +/** + * Used to read and write data stored in files to/from the [[InternalRow]] format. + */ +trait FileFormat { + /** + * When possible, this method should return the schema of the given `files`. When the format + * does not support inference, or no valid files are given should return None. In these cases + * Spark will require that user specify the schema manually. + */ + def inferSchema( + sqlContext: SQLContext, + options: Map[String, String], + files: Seq[FileStatus]): Option[StructType] + + /** + * Prepares a read job and returns a potentially updated data source option [[Map]]. This method + * can be useful for collecting necessary global information for scanning input data. + */ + def prepareRead( + sqlContext: SQLContext, + options: Map[String, String], + files: Seq[FileStatus]): Map[String, String] = options + + /** + * Prepares a write job and returns an [[OutputWriterFactory]]. Client side job preparation can + * be put here. For example, user defined output committer can be configured here + * by setting the output committer class in the conf of spark.sql.sources.outputCommitterClass. + */ + def prepareWrite( + sqlContext: SQLContext, + job: Job, + options: Map[String, String], + dataSchema: StructType): OutputWriterFactory + + /** + * Returns whether this format support returning columnar batch or not. + * + * TODO: we should just have different traits for the different formats. + */ + def supportBatch(sqlContext: SQLContext, dataSchema: StructType): Boolean = { + false + } + + /** + * Returns a function that can be used to read a single file in as an Iterator of InternalRow. + * + * @param dataSchema The global data schema. It can be either specified by the user, or + * reconciled/merged from all underlying data files. If any partition columns + * are contained in the files, they are preserved in this schema. + * @param partitionSchema The schema of the partition column row that will be present in each + * PartitionedFile. These columns should be appended to the rows that + * are produced by the iterator. + * @param requiredSchema The schema of the data that should be output for each row. This may be a + * subset of the columns that are present in the file if column pruning has + * occurred. + * @param filters A set of filters than can optionally be used to reduce the number of rows output + * @param options A set of string -> string configuration options. + * @return + */ + def buildReader( + sqlContext: SQLContext, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String]): PartitionedFile => Iterator[InternalRow] = { + // TODO: Remove this default implementation when the other formats have been ported + // Until then we guard in [[FileSourceStrategy]] to only call this method on supported formats. + throw new UnsupportedOperationException(s"buildReader is not supported for $this") + } +} + +/** + * A collection of data files from a partitioned relation, along with the partition values in the + * form of an [[InternalRow]]. + */ +case class Partition(values: InternalRow, files: Seq[FileStatus]) + +/** + * An interface for objects capable of enumerating the files that comprise a relation as well + * as the partitioning characteristics of those files. + */ +trait FileCatalog { + def paths: Seq[Path] + + def partitionSpec(): PartitionSpec + + /** + * Returns all valid files grouped into partitions when the data is partitioned. If the data is + * unpartitioned, this will return a single partition with not partition values. + * + * @param filters the filters used to prune which partitions are returned. These filters must + * only refer to partition columns and this method will only return files + * where these predicates are guaranteed to evaluate to `true`. Thus, these + * filters will not need to be evaluated again on the returned data. + */ + def listFiles(filters: Seq[Expression]): Seq[Partition] + + def allFiles(): Seq[FileStatus] + + def getStatus(path: Path): Array[FileStatus] + + def refresh(): Unit +} + +/** + * A file catalog that caches metadata gathered by scanning all the files present in `paths` + * recursively. + * + * @param parameters as set of options to control discovery + * @param paths a list of paths to scan + * @param partitionSchema an optional partition schema that will be use to provide types for the + * discovered partitions + */ +class HDFSFileCatalog( + val sqlContext: SQLContext, + val parameters: Map[String, String], + val paths: Seq[Path], + val partitionSchema: Option[StructType]) + extends FileCatalog with Logging { + + private val hadoopConf = new Configuration(sqlContext.sparkContext.hadoopConfiguration) + + var leafFiles = mutable.LinkedHashMap.empty[Path, FileStatus] + var leafDirToChildrenFiles = mutable.Map.empty[Path, Array[FileStatus]] + var cachedPartitionSpec: PartitionSpec = _ + + def partitionSpec(): PartitionSpec = { + if (cachedPartitionSpec == null) { + cachedPartitionSpec = inferPartitioning(partitionSchema) + } + + cachedPartitionSpec + } + + refresh() + + override def listFiles(filters: Seq[Expression]): Seq[Partition] = { + if (partitionSpec().partitionColumns.isEmpty) { + Partition(InternalRow.empty, allFiles().filterNot(_.getPath.getName startsWith "_")) :: Nil + } else { + prunePartitions(filters, partitionSpec()).map { + case PartitionDirectory(values, path) => + Partition( + values, + getStatus(path).filterNot(_.getPath.getName startsWith "_")) + } + } + } + + protected def prunePartitions( + predicates: Seq[Expression], + partitionSpec: PartitionSpec): Seq[PartitionDirectory] = { + val PartitionSpec(partitionColumns, partitions) = partitionSpec + val partitionColumnNames = partitionColumns.map(_.name).toSet + val partitionPruningPredicates = predicates.filter { + _.references.map(_.name).toSet.subsetOf(partitionColumnNames) + } + + if (partitionPruningPredicates.nonEmpty) { + val predicate = partitionPruningPredicates.reduce(expressions.And) + + val boundPredicate = InterpretedPredicate.create(predicate.transform { + case a: AttributeReference => + val index = partitionColumns.indexWhere(a.name == _.name) + BoundReference(index, partitionColumns(index).dataType, nullable = true) + }) + + val selected = partitions.filter { + case PartitionDirectory(values, _) => boundPredicate(values) + } + logInfo { + val total = partitions.length + val selectedSize = selected.length + val percentPruned = (1 - selectedSize.toDouble / total.toDouble) * 100 + s"Selected $selectedSize partitions out of $total, pruned $percentPruned% partitions." + } + + selected + } else { + partitions + } + } + + def allFiles(): Seq[FileStatus] = leafFiles.values.toSeq + + def getStatus(path: Path): Array[FileStatus] = leafDirToChildrenFiles(path) + + private def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = { + if (paths.length >= sqlContext.conf.parallelPartitionDiscoveryThreshold) { + HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, sqlContext.sparkContext) + } else { + val statuses = paths.flatMap { path => + val fs = path.getFileSystem(hadoopConf) + logInfo(s"Listing $path on driver") + // Dummy jobconf to get to the pathFilter defined in configuration + val jobConf = new JobConf(hadoopConf, this.getClass()) + val pathFilter = FileInputFormat.getInputPathFilter(jobConf) + if (pathFilter != null) { + Try(fs.listStatus(path, pathFilter)).getOrElse(Array.empty) + } else { + Try(fs.listStatus(path)).getOrElse(Array.empty) + } + }.filterNot { status => + val name = status.getPath.getName + HadoopFsRelation.shouldFilterOut(name) + } + + val (dirs, files) = statuses.partition(_.isDirectory) + + // It uses [[LinkedHashSet]] since the order of files can affect the results. (SPARK-11500) + if (dirs.isEmpty) { + mutable.LinkedHashSet(files: _*) + } else { + mutable.LinkedHashSet(files: _*) ++ listLeafFiles(dirs.map(_.getPath)) + } + } + } + + def inferPartitioning(schema: Option[StructType]): PartitionSpec = { + // We use leaf dirs containing data files to discover the schema. + val leafDirs = leafDirToChildrenFiles.keys.toSeq + schema match { + case Some(userProvidedSchema) if userProvidedSchema.nonEmpty => + val spec = PartitioningUtils.parsePartitions( + leafDirs, + PartitioningUtils.DEFAULT_PARTITION_NAME, + typeInference = false, + basePaths = basePaths) + + // Without auto inference, all of value in the `row` should be null or in StringType, + // we need to cast into the data type that user specified. + def castPartitionValuesToUserSchema(row: InternalRow) = { + InternalRow((0 until row.numFields).map { i => + Cast( + Literal.create(row.getUTF8String(i), StringType), + userProvidedSchema.fields(i).dataType).eval() + }: _*) + } + + PartitionSpec(userProvidedSchema, spec.partitions.map { part => + part.copy(values = castPartitionValuesToUserSchema(part.values)) + }) + case _ => + PartitioningUtils.parsePartitions( + leafDirs, + PartitioningUtils.DEFAULT_PARTITION_NAME, + typeInference = sqlContext.conf.partitionColumnTypeInferenceEnabled(), + basePaths = basePaths) + } + } + + /** + * Contains a set of paths that are considered as the base dirs of the input datasets. + * The partitioning discovery logic will make sure it will stop when it reaches any + * base path. By default, the paths of the dataset provided by users will be base paths. + * For example, if a user uses `sqlContext.read.parquet("/path/something=true/")`, the base path + * will be `/path/something=true/`, and the returned DataFrame will not contain a column of + * `something`. If users want to override the basePath. They can set `basePath` in the options + * to pass the new base path to the data source. + * For the above example, if the user-provided base path is `/path/`, the returned + * DataFrame will have the column of `something`. + */ + private def basePaths: Set[Path] = { + val userDefinedBasePath = parameters.get("basePath").map(basePath => Set(new Path(basePath))) + userDefinedBasePath.getOrElse { + // If the user does not provide basePath, we will just use paths. + paths.toSet + }.map { hdfsPath => + // Make the path qualified (consistent with listLeafFiles and listLeafFilesInParallel). + val fs = hdfsPath.getFileSystem(hadoopConf) + hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory) + } + } + + def refresh(): Unit = { + val files = listLeafFiles(paths) + + leafFiles.clear() + leafDirToChildrenFiles.clear() + + leafFiles ++= files.map(f => f.getPath -> f) + leafDirToChildrenFiles ++= files.toArray.groupBy(_.getPath.getParent) + + cachedPartitionSpec = null + } + + override def equals(other: Any): Boolean = other match { + case hdfs: HDFSFileCatalog => paths.toSet == hdfs.paths.toSet + case _ => false + } + + override def hashCode(): Int = paths.toSet.hashCode() +} + +/** + * Helper methods for gathering metadata from HDFS. + */ +private[sql] object HadoopFsRelation extends Logging { + + /** Checks if we should filter out this path name. */ + def shouldFilterOut(pathName: String): Boolean = { + // TODO: We should try to filter out all files/dirs starting with "." or "_". + // The only reason that we are not doing it now is that Parquet needs to find those + // metadata files from leaf files returned by this methods. We should refactor + // this logic to not mix metadata files with data files. + pathName == "_SUCCESS" || pathName == "_temporary" || pathName.startsWith(".") + } + + // We don't filter files/directories whose name start with "_" except "_temporary" here, as + // specific data sources may take advantages over them (e.g. Parquet _metadata and + // _common_metadata files). "_temporary" directories are explicitly ignored since failed + // tasks/jobs may leave partial/corrupted data files there. Files and directories whose name + // start with "." are also ignored. + def listLeafFiles(fs: FileSystem, status: FileStatus): Array[FileStatus] = { + logInfo(s"Listing ${status.getPath}") + val name = status.getPath.getName.toLowerCase + if (shouldFilterOut(name)) { + Array.empty + } else { + // Dummy jobconf to get to the pathFilter defined in configuration + val jobConf = new JobConf(fs.getConf, this.getClass()) + val pathFilter = FileInputFormat.getInputPathFilter(jobConf) + val statuses = + if (pathFilter != null) { + val (dirs, files) = fs.listStatus(status.getPath, pathFilter).partition(_.isDirectory) + files ++ dirs.flatMap(dir => listLeafFiles(fs, dir)) + } else { + val (dirs, files) = fs.listStatus(status.getPath).partition(_.isDirectory) + files ++ dirs.flatMap(dir => listLeafFiles(fs, dir)) + } + statuses.filterNot(status => shouldFilterOut(status.getPath.getName)) + } + } + + // `FileStatus` is Writable but not serializable. What make it worse, somehow it doesn't play + // well with `SerializableWritable`. So there seems to be no way to serialize a `FileStatus`. + // Here we use `FakeFileStatus` to extract key components of a `FileStatus` to serialize it from + // executor side and reconstruct it on driver side. + case class FakeFileStatus( + path: String, + length: Long, + isDir: Boolean, + blockReplication: Short, + blockSize: Long, + modificationTime: Long, + accessTime: Long) + + def listLeafFilesInParallel( + paths: Seq[Path], + hadoopConf: Configuration, + sparkContext: SparkContext): mutable.LinkedHashSet[FileStatus] = { + logInfo(s"Listing leaf files and directories in parallel under: ${paths.mkString(", ")}") + + val serializableConfiguration = new SerializableConfiguration(hadoopConf) + val serializedPaths = paths.map(_.toString) + + val fakeStatuses = sparkContext.parallelize(serializedPaths).map(new Path(_)).flatMap { path => + val fs = path.getFileSystem(serializableConfiguration.value) + Try(listLeafFiles(fs, fs.getFileStatus(path))).getOrElse(Array.empty) + }.map { status => + FakeFileStatus( + status.getPath.toString, + status.getLen, + status.isDirectory, + status.getReplication, + status.getBlockSize, + status.getModificationTime, + status.getAccessTime) + }.collect() + + val hadoopFakeStatuses = fakeStatuses.map { f => + new FileStatus( + f.length, f.isDir, f.blockReplication, f.blockSize, f.modificationTime, new Path(f.path)) + } + mutable.LinkedHashSet(hadoopFakeStatuses: _*) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 28ac4583e9b25..5b8dc4a3ee723 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.{BaseRelation, HadoopFsRelation, InsertableRelation} +import org.apache.spark.sql.sources.{BaseRelation, InsertableRelation} /** * Try to replaces [[UnresolvedRelation]]s with [[ResolvedDataSource]]. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala index 94ecb7a28663c..fa0df61ca5f2c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.{AnalysisException, Row, SQLContext} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.catalyst.expressions.codegen.{BufferHolder, UnsafeRowWriter} -import org.apache.spark.sql.execution.datasources.{CompressionCodecs, HadoopFileLinesReader, PartitionedFile} +import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala index 6921ae584dd84..f3c1cc5ef5ed3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, SQLContext} -import org.apache.spark.sql.sources.FileFormat +import org.apache.spark.sql.execution.datasources.FileFormat object FileStreamSink { // The name of the subdirectory that is used to store metadata about which files are valid. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamFileCatalog.scala index b8d69b18450cf..b1f93a9159d50 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamFileCatalog.scala @@ -23,8 +23,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.execution.datasources.PartitionSpec -import org.apache.spark.sql.sources.{FileCatalog, Partition} +import org.apache.spark.sql.execution.datasources.{FileCatalog, Partition, PartitionSpec} import org.apache.spark.sql.types.StructType class StreamFileCatalog(sqlContext: SQLContext, path: Path) extends FileCatalog with Logging { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 4b9bf8daae37c..26285bde31ad0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -17,28 +17,13 @@ package org.apache.spark.sql.sources -import scala.collection.mutable -import scala.util.Try - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} -import org.apache.hadoop.mapred.{FileInputFormat, JobConf} -import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} - -import org.apache.spark.SparkContext import org.apache.spark.annotation.{DeveloperApi, Experimental} -import org.apache.spark.broadcast.Broadcast -import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.{expressions, CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.execution.FileRelation -import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.streaming.{Sink, Source} -import org.apache.spark.sql.types.{StringType, StructType} -import org.apache.spark.util.SerializableConfiguration -import org.apache.spark.util.collection.BitSet +import org.apache.spark.sql.types.StructType /** * ::DeveloperApi:: @@ -318,496 +303,3 @@ trait InsertableRelation { trait CatalystScan { def buildScan(requiredColumns: Seq[Attribute], filters: Seq[Expression]): RDD[Row] } - -/** - * ::Experimental:: - * A factory that produces [[OutputWriter]]s. A new [[OutputWriterFactory]] is created on driver - * side for each write job issued when writing to a [[HadoopFsRelation]], and then gets serialized - * to executor side to create actual [[OutputWriter]]s on the fly. - * - * @since 1.4.0 - */ -@Experimental -abstract class OutputWriterFactory extends Serializable { - /** - * When writing to a [[HadoopFsRelation]], this method gets called by each task on executor side - * to instantiate new [[OutputWriter]]s. - * - * @param path Path of the file to which this [[OutputWriter]] is supposed to write. Note that - * this may not point to the final output file. For example, `FileOutputFormat` writes to - * temporary directories and then merge written files back to the final destination. In - * this case, `path` points to a temporary output file under the temporary directory. - * @param dataSchema Schema of the rows to be written. Partition columns are not included in the - * schema if the relation being written is partitioned. - * @param context The Hadoop MapReduce task context. - * @since 1.4.0 - */ - private[sql] def newInstance( - path: String, - bucketId: Option[Int], // TODO: This doesn't belong here... - dataSchema: StructType, - context: TaskAttemptContext): OutputWriter -} - -/** - * ::Experimental:: - * [[OutputWriter]] is used together with [[HadoopFsRelation]] for persisting rows to the - * underlying file system. Subclasses of [[OutputWriter]] must provide a zero-argument constructor. - * An [[OutputWriter]] instance is created and initialized when a new output file is opened on - * executor side. This instance is used to persist rows to this single output file. - * - * @since 1.4.0 - */ -@Experimental -abstract class OutputWriter { - /** - * Persists a single row. Invoked on the executor side. When writing to dynamically partitioned - * tables, dynamic partition columns are not included in rows to be written. - * - * @since 1.4.0 - */ - def write(row: Row): Unit - - /** - * Closes the [[OutputWriter]]. Invoked on the executor side after all rows are persisted, before - * the task output is committed. - * - * @since 1.4.0 - */ - def close(): Unit - - private var converter: InternalRow => Row = _ - - protected[sql] def initConverter(dataSchema: StructType) = { - converter = - CatalystTypeConverters.createToScalaConverter(dataSchema).asInstanceOf[InternalRow => Row] - } - - protected[sql] def writeInternal(row: InternalRow): Unit = { - write(converter(row)) - } -} - -/** - * Acts as a container for all of the metadata required to read from a datasource. All discovery, - * resolution and merging logic for schemas and partitions has been removed. - * - * @param location A [[FileCatalog]] that can enumerate the locations of all the files that comprise - * this relation. - * @param partitionSchema The schema of the columns (if any) that are used to partition the relation - * @param dataSchema The schema of any remaining columns. Note that if any partition columns are - * present in the actual data files as well, they are preserved. - * @param bucketSpec Describes the bucketing (hash-partitioning of the files by some column values). - * @param fileFormat A file format that can be used to read and write the data in files. - * @param options Configuration used when reading / writing data. - */ -case class HadoopFsRelation( - sqlContext: SQLContext, - location: FileCatalog, - partitionSchema: StructType, - dataSchema: StructType, - bucketSpec: Option[BucketSpec], - fileFormat: FileFormat, - options: Map[String, String]) extends BaseRelation with FileRelation { - - val schema: StructType = { - val dataSchemaColumnNames = dataSchema.map(_.name.toLowerCase).toSet - StructType(dataSchema ++ partitionSchema.filterNot { column => - dataSchemaColumnNames.contains(column.name.toLowerCase) - }) - } - - def partitionSchemaOption: Option[StructType] = - if (partitionSchema.isEmpty) None else Some(partitionSchema) - def partitionSpec: PartitionSpec = location.partitionSpec() - - def refresh(): Unit = location.refresh() - - override def toString: String = - s"HadoopFiles" - - /** Returns the list of files that will be read when scanning this relation. */ - override def inputFiles: Array[String] = - location.allFiles().map(_.getPath.toUri.toString).toArray - - override def sizeInBytes: Long = location.allFiles().map(_.getLen).sum -} - -/** - * Used to read and write data stored in files to/from the [[InternalRow]] format. - */ -trait FileFormat { - /** - * When possible, this method should return the schema of the given `files`. When the format - * does not support inference, or no valid files are given should return None. In these cases - * Spark will require that user specify the schema manually. - */ - def inferSchema( - sqlContext: SQLContext, - options: Map[String, String], - files: Seq[FileStatus]): Option[StructType] - - /** - * Prepares a read job and returns a potentially updated data source option [[Map]]. This method - * can be useful for collecting necessary global information for scanning input data. - */ - def prepareRead( - sqlContext: SQLContext, - options: Map[String, String], - files: Seq[FileStatus]): Map[String, String] = options - - /** - * Prepares a write job and returns an [[OutputWriterFactory]]. Client side job preparation can - * be put here. For example, user defined output committer can be configured here - * by setting the output committer class in the conf of spark.sql.sources.outputCommitterClass. - */ - def prepareWrite( - sqlContext: SQLContext, - job: Job, - options: Map[String, String], - dataSchema: StructType): OutputWriterFactory - - /** - * Returns whether this format support returning columnar batch or not. - * - * TODO: we should just have different traits for the different formats. - */ - def supportBatch(sqlContext: SQLContext, dataSchema: StructType): Boolean = { - false - } - - /** - * Returns a function that can be used to read a single file in as an Iterator of InternalRow. - * - * @param dataSchema The global data schema. It can be either specified by the user, or - * reconciled/merged from all underlying data files. If any partition columns - * are contained in the files, they are preserved in this schema. - * @param partitionSchema The schema of the partition column row that will be present in each - * PartitionedFile. These columns should be appended to the rows that - * are produced by the iterator. - * @param requiredSchema The schema of the data that should be output for each row. This may be a - * subset of the columns that are present in the file if column pruning has - * occurred. - * @param filters A set of filters than can optionally be used to reduce the number of rows output - * @param options A set of string -> string configuration options. - * @return - */ - def buildReader( - sqlContext: SQLContext, - dataSchema: StructType, - partitionSchema: StructType, - requiredSchema: StructType, - filters: Seq[Filter], - options: Map[String, String]): PartitionedFile => Iterator[InternalRow] = { - // TODO: Remove this default implementation when the other formats have been ported - // Until then we guard in [[FileSourceStrategy]] to only call this method on supported formats. - throw new UnsupportedOperationException(s"buildReader is not supported for $this") - } -} - -/** - * A collection of data files from a partitioned relation, along with the partition values in the - * form of an [[InternalRow]]. - */ -case class Partition(values: InternalRow, files: Seq[FileStatus]) - -/** - * An interface for objects capable of enumerating the files that comprise a relation as well - * as the partitioning characteristics of those files. - */ -trait FileCatalog { - def paths: Seq[Path] - - def partitionSpec(): PartitionSpec - - /** - * Returns all valid files grouped into partitions when the data is partitioned. If the data is - * unpartitioned, this will return a single partition with not partition values. - * - * @param filters the filters used to prune which partitions are returned. These filters must - * only refer to partition columns and this method will only return files - * where these predicates are guaranteed to evaluate to `true`. Thus, these - * filters will not need to be evaluated again on the returned data. - */ - def listFiles(filters: Seq[Expression]): Seq[Partition] - - def allFiles(): Seq[FileStatus] - - def getStatus(path: Path): Array[FileStatus] - - def refresh(): Unit -} - -/** - * A file catalog that caches metadata gathered by scanning all the files present in `paths` - * recursively. - * - * @param parameters as set of options to control discovery - * @param paths a list of paths to scan - * @param partitionSchema an optional partition schema that will be use to provide types for the - * discovered partitions - */ -class HDFSFileCatalog( - val sqlContext: SQLContext, - val parameters: Map[String, String], - val paths: Seq[Path], - val partitionSchema: Option[StructType]) - extends FileCatalog with Logging { - - private val hadoopConf = new Configuration(sqlContext.sparkContext.hadoopConfiguration) - - var leafFiles = mutable.LinkedHashMap.empty[Path, FileStatus] - var leafDirToChildrenFiles = mutable.Map.empty[Path, Array[FileStatus]] - var cachedPartitionSpec: PartitionSpec = _ - - def partitionSpec(): PartitionSpec = { - if (cachedPartitionSpec == null) { - cachedPartitionSpec = inferPartitioning(partitionSchema) - } - - cachedPartitionSpec - } - - refresh() - - override def listFiles(filters: Seq[Expression]): Seq[Partition] = { - if (partitionSpec().partitionColumns.isEmpty) { - Partition(InternalRow.empty, allFiles().filterNot(_.getPath.getName startsWith "_")) :: Nil - } else { - prunePartitions(filters, partitionSpec()).map { - case PartitionDirectory(values, path) => - Partition( - values, - getStatus(path).filterNot(_.getPath.getName startsWith "_")) - } - } - } - - protected def prunePartitions( - predicates: Seq[Expression], - partitionSpec: PartitionSpec): Seq[PartitionDirectory] = { - val PartitionSpec(partitionColumns, partitions) = partitionSpec - val partitionColumnNames = partitionColumns.map(_.name).toSet - val partitionPruningPredicates = predicates.filter { - _.references.map(_.name).toSet.subsetOf(partitionColumnNames) - } - - if (partitionPruningPredicates.nonEmpty) { - val predicate = partitionPruningPredicates.reduce(expressions.And) - - val boundPredicate = InterpretedPredicate.create(predicate.transform { - case a: AttributeReference => - val index = partitionColumns.indexWhere(a.name == _.name) - BoundReference(index, partitionColumns(index).dataType, nullable = true) - }) - - val selected = partitions.filter { - case PartitionDirectory(values, _) => boundPredicate(values) - } - logInfo { - val total = partitions.length - val selectedSize = selected.length - val percentPruned = (1 - selectedSize.toDouble / total.toDouble) * 100 - s"Selected $selectedSize partitions out of $total, pruned $percentPruned% partitions." - } - - selected - } else { - partitions - } - } - - def allFiles(): Seq[FileStatus] = leafFiles.values.toSeq - - def getStatus(path: Path): Array[FileStatus] = leafDirToChildrenFiles(path) - - private def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = { - if (paths.length >= sqlContext.conf.parallelPartitionDiscoveryThreshold) { - HadoopFsRelation.listLeafFilesInParallel(paths, hadoopConf, sqlContext.sparkContext) - } else { - val statuses = paths.flatMap { path => - val fs = path.getFileSystem(hadoopConf) - logInfo(s"Listing $path on driver") - // Dummy jobconf to get to the pathFilter defined in configuration - val jobConf = new JobConf(hadoopConf, this.getClass()) - val pathFilter = FileInputFormat.getInputPathFilter(jobConf) - if (pathFilter != null) { - Try(fs.listStatus(path, pathFilter)).getOrElse(Array.empty) - } else { - Try(fs.listStatus(path)).getOrElse(Array.empty) - } - }.filterNot { status => - val name = status.getPath.getName - HadoopFsRelation.shouldFilterOut(name) - } - - val (dirs, files) = statuses.partition(_.isDirectory) - - // It uses [[LinkedHashSet]] since the order of files can affect the results. (SPARK-11500) - if (dirs.isEmpty) { - mutable.LinkedHashSet(files: _*) - } else { - mutable.LinkedHashSet(files: _*) ++ listLeafFiles(dirs.map(_.getPath)) - } - } - } - - def inferPartitioning(schema: Option[StructType]): PartitionSpec = { - // We use leaf dirs containing data files to discover the schema. - val leafDirs = leafDirToChildrenFiles.keys.toSeq - schema match { - case Some(userProvidedSchema) if userProvidedSchema.nonEmpty => - val spec = PartitioningUtils.parsePartitions( - leafDirs, - PartitioningUtils.DEFAULT_PARTITION_NAME, - typeInference = false, - basePaths = basePaths) - - // Without auto inference, all of value in the `row` should be null or in StringType, - // we need to cast into the data type that user specified. - def castPartitionValuesToUserSchema(row: InternalRow) = { - InternalRow((0 until row.numFields).map { i => - Cast( - Literal.create(row.getUTF8String(i), StringType), - userProvidedSchema.fields(i).dataType).eval() - }: _*) - } - - PartitionSpec(userProvidedSchema, spec.partitions.map { part => - part.copy(values = castPartitionValuesToUserSchema(part.values)) - }) - case _ => - PartitioningUtils.parsePartitions( - leafDirs, - PartitioningUtils.DEFAULT_PARTITION_NAME, - typeInference = sqlContext.conf.partitionColumnTypeInferenceEnabled(), - basePaths = basePaths) - } - } - - /** - * Contains a set of paths that are considered as the base dirs of the input datasets. - * The partitioning discovery logic will make sure it will stop when it reaches any - * base path. By default, the paths of the dataset provided by users will be base paths. - * For example, if a user uses `sqlContext.read.parquet("/path/something=true/")`, the base path - * will be `/path/something=true/`, and the returned DataFrame will not contain a column of - * `something`. If users want to override the basePath. They can set `basePath` in the options - * to pass the new base path to the data source. - * For the above example, if the user-provided base path is `/path/`, the returned - * DataFrame will have the column of `something`. - */ - private def basePaths: Set[Path] = { - val userDefinedBasePath = parameters.get("basePath").map(basePath => Set(new Path(basePath))) - userDefinedBasePath.getOrElse { - // If the user does not provide basePath, we will just use paths. - paths.toSet - }.map { hdfsPath => - // Make the path qualified (consistent with listLeafFiles and listLeafFilesInParallel). - val fs = hdfsPath.getFileSystem(hadoopConf) - hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory) - } - } - - def refresh(): Unit = { - val files = listLeafFiles(paths) - - leafFiles.clear() - leafDirToChildrenFiles.clear() - - leafFiles ++= files.map(f => f.getPath -> f) - leafDirToChildrenFiles ++= files.toArray.groupBy(_.getPath.getParent) - - cachedPartitionSpec = null - } - - override def equals(other: Any): Boolean = other match { - case hdfs: HDFSFileCatalog => paths.toSet == hdfs.paths.toSet - case _ => false - } - - override def hashCode(): Int = paths.toSet.hashCode() -} - -/** - * Helper methods for gathering metadata from HDFS. - */ -private[sql] object HadoopFsRelation extends Logging { - - /** Checks if we should filter out this path name. */ - def shouldFilterOut(pathName: String): Boolean = { - // TODO: We should try to filter out all files/dirs starting with "." or "_". - // The only reason that we are not doing it now is that Parquet needs to find those - // metadata files from leaf files returned by this methods. We should refactor - // this logic to not mix metadata files with data files. - pathName == "_SUCCESS" || pathName == "_temporary" || pathName.startsWith(".") - } - - // We don't filter files/directories whose name start with "_" except "_temporary" here, as - // specific data sources may take advantages over them (e.g. Parquet _metadata and - // _common_metadata files). "_temporary" directories are explicitly ignored since failed - // tasks/jobs may leave partial/corrupted data files there. Files and directories whose name - // start with "." are also ignored. - def listLeafFiles(fs: FileSystem, status: FileStatus): Array[FileStatus] = { - logInfo(s"Listing ${status.getPath}") - val name = status.getPath.getName.toLowerCase - if (shouldFilterOut(name)) { - Array.empty - } else { - // Dummy jobconf to get to the pathFilter defined in configuration - val jobConf = new JobConf(fs.getConf, this.getClass()) - val pathFilter = FileInputFormat.getInputPathFilter(jobConf) - val statuses = - if (pathFilter != null) { - val (dirs, files) = fs.listStatus(status.getPath, pathFilter).partition(_.isDirectory) - files ++ dirs.flatMap(dir => listLeafFiles(fs, dir)) - } else { - val (dirs, files) = fs.listStatus(status.getPath).partition(_.isDirectory) - files ++ dirs.flatMap(dir => listLeafFiles(fs, dir)) - } - statuses.filterNot(status => shouldFilterOut(status.getPath.getName)) - } - } - - // `FileStatus` is Writable but not serializable. What make it worse, somehow it doesn't play - // well with `SerializableWritable`. So there seems to be no way to serialize a `FileStatus`. - // Here we use `FakeFileStatus` to extract key components of a `FileStatus` to serialize it from - // executor side and reconstruct it on driver side. - case class FakeFileStatus( - path: String, - length: Long, - isDir: Boolean, - blockReplication: Short, - blockSize: Long, - modificationTime: Long, - accessTime: Long) - - def listLeafFilesInParallel( - paths: Seq[Path], - hadoopConf: Configuration, - sparkContext: SparkContext): mutable.LinkedHashSet[FileStatus] = { - logInfo(s"Listing leaf files and directories in parallel under: ${paths.mkString(", ")}") - - val serializableConfiguration = new SerializableConfiguration(hadoopConf) - val serializedPaths = paths.map(_.toString) - - val fakeStatuses = sparkContext.parallelize(serializedPaths).map(new Path(_)).flatMap { path => - val fs = path.getFileSystem(serializableConfiguration.value) - Try(listLeafFiles(fs, fs.getFileStatus(path))).getOrElse(Array.empty) - }.map { status => - FakeFileStatus( - status.getPath.toString, - status.getLen, - status.isDirectory, - status.getReplication, - status.getBlockSize, - status.getModificationTime, - status.getAccessTime) - }.collect() - - val hadoopFakeStatuses = fakeStatuses.map { f => - new FileStatus( - f.length, f.isDir, f.blockReplication, f.blockSize, f.modificationTime, new Path(f.path)) - } - mutable.LinkedHashSet(hadoopFakeStatuses: _*) - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index 51183e970d965..65635e3c066d9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -27,10 +27,9 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, LogicalRelation} +import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.HadoopFsRelation import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index f875b54cd6649..5bffb307ec80e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -29,9 +29,8 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Literal -import org.apache.spark.sql.execution.datasources.{LogicalRelation, PartitionDirectory => Partition, PartitioningUtils, PartitionSpec} +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PartitionDirectory => Partition, PartitioningUtils, PartitionSpec} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.HadoopFsRelation import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 79fe23b258c19..753950ff844c9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -41,12 +41,11 @@ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.execution.FileRelation -import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.{Partition => _, _} import org.apache.spark.sql.execution.datasources.parquet.{DefaultSource => ParquetDefaultSource, ParquetRelation} import org.apache.spark.sql.hive.client._ import org.apache.spark.sql.hive.execution.HiveNativeCommand import org.apache.spark.sql.hive.orc.{DefaultSource => OrcDefaultSource} -import org.apache.spark.sql.sources.{FileFormat, HadoopFsRelation, HDFSFileCatalog} import org.apache.spark.sql.types._ private[hive] case class HiveSerDe( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 8f7c4e8289900..7a2b60dde5a3d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.command.RunnableCommand -import org.apache.spark.sql.execution.datasources.{BucketSpec, DataSource, LogicalRelation} +import org.apache.spark.sql.execution.datasources.{BucketSpec, DataSource, HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation} import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 3c299daa778cc..d0e687051914e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -26,10 +26,9 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} -import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.HadoopFsRelation import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ import org.apache.spark.util.Utils diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 5ce16be4dc059..1098e74cab684 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -22,12 +22,11 @@ import java.sql.{Date, Timestamp} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, FunctionRegistry} -import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.HadoopFsRelation import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala index 7b0c7a9f00514..ddabab3a14b51 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala @@ -27,8 +27,7 @@ import org.apache.spark.sql.{Column, DataFrame, QueryTest} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, LogicalRelation} -import org.apache.spark.sql.sources.HadoopFsRelation +import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} /** * A test suite that tests ORC filter API based filter pushdown optimization. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index eac65d5720575..4b2b1a160ad5c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -23,11 +23,10 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.DataSourceScan import org.apache.spark.sql.execution.command.ExecutedCommand -import org.apache.spark.sql.execution.datasources.{InsertIntoDataSource, InsertIntoHadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InsertIntoDataSource, InsertIntoHadoopFsRelation, LogicalRelation} import org.apache.spark.sql.hive.execution.HiveTableScan import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.HadoopFsRelation import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ import org.apache.spark.util.Utils diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala index 6d7e7b77df81e..4a2d1903533b2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala @@ -21,11 +21,14 @@ import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} import org.apache.spark.TaskContext import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.execution.datasources.{OutputWriter, OutputWriterFactory} import org.apache.spark.sql.types.StructType class CommitFailureTestSource extends SimpleTextSource { /** - * Prepares a write job and returns an [[OutputWriterFactory]]. Client side job preparation can + * Prepares a write job and returns an + * [[org.apache.spark.sql.execution.datasources.OutputWriterFactory]]. + * Client side job preparation can * be put here. For example, user defined output committer can be configured here * by setting the output committer class in the conf of spark.sql.sources.outputCommitterClass. */ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala index 113b124be30d9..16996ae036671 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.{sources, Row, SQLContext} import org.apache.spark.sql.catalyst.{expressions, InternalRow} import org.apache.spark.sql.catalyst.expressions.{Cast, Expression, GenericInternalRow, InterpretedPredicate, InterpretedProjection, JoinedRow, Literal} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection -import org.apache.spark.sql.execution.datasources.{HadoopFileLinesReader, PartitionedFile} +import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.util.SerializableConfiguration diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala index 10eeb30242e2c..3d0225679205a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala @@ -28,7 +28,7 @@ import org.apache.parquet.hadoop.ParquetOutputCommitter import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.sql._ -import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils From 3ae25f244bd471ef77002c703f2cc7ed6b524f11 Mon Sep 17 00:00:00 2001 From: Joan Date: Tue, 19 Apr 2016 17:36:31 -0700 Subject: [PATCH 42/69] [SPARK-13929] Use Scala reflection for UDTs ## What changes were proposed in this pull request? Enable ScalaReflection and User Defined Types for plain Scala classes. This involves the move of `schemaFor` from `ScalaReflection` trait (which is Runtime and Compile time (macros) reflection) to the `ScalaReflection` object (runtime reflection only) as I believe this code wouldn't work at compile time anyway as it manipulates `Class`'s that are not compiled yet. ## How was this patch tested? Unit test Author: Joan Closes #12149 from joan38/SPARK-13929-Scala-reflection. --- .../spark/sql/types/SQLUserDefinedType.java | 5 - .../spark/sql/catalyst/ScalaReflection.scala | 98 ++++++-------- .../sql/catalyst/ScalaReflectionSuite.scala | 37 +++++- .../spark/sql/UserDefinedTypeSuite.scala | 123 +++++++++++------- .../datasources/json/JsonSuite.scala | 4 +- .../execution/AggregationQuerySuite.scala | 2 +- .../sql/sources/hadoopFsRelationSuites.scala | 2 +- 7 files changed, 157 insertions(+), 114 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/types/SQLUserDefinedType.java b/sql/catalyst/src/main/java/org/apache/spark/sql/types/SQLUserDefinedType.java index 1e4e5ede8cc11..110ed460cc8fa 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/types/SQLUserDefinedType.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/types/SQLUserDefinedType.java @@ -24,11 +24,6 @@ /** * ::DeveloperApi:: * A user-defined type which can be automatically recognized by a SQLContext and registered. - *

- * WARNING: This annotation will only work if both Java and Scala reflection return the same class - * names (after erasure) for the UDT. This will NOT be the case when, e.g., the UDT class - * is enclosed in an object (a singleton). - *

* WARNING: UDTs are currently only supported from Scala. */ // TODO: Should I used @Documented ? diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 4795fc25576aa..bd723135b510d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -374,10 +374,8 @@ object ScalaReflection extends ScalaReflection { newInstance } - case t if Utils.classIsLoadable(className) && - Utils.classForName(className).isAnnotationPresent(classOf[SQLUserDefinedType]) => - val udt = Utils.classForName(className) - .getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance() + case t if t.typeSymbol.annotations.exists(_.tpe =:= typeOf[SQLUserDefinedType]) => + val udt = getClassFromType(t).getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance() val obj = NewInstance( udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt(), Nil, @@ -432,7 +430,6 @@ object ScalaReflection extends ScalaReflection { if (!inputObject.dataType.isInstanceOf[ObjectType]) { inputObject } else { - val className = getClassNameFromType(tpe) tpe match { case t if t <:< localTypeOf[Option[_]] => val TypeRef(_, _, Seq(optType)) = t @@ -589,9 +586,8 @@ object ScalaReflection extends ScalaReflection { case t if t <:< localTypeOf[java.lang.Boolean] => Invoke(inputObject, "booleanValue", BooleanType) - case t if Utils.classIsLoadable(className) && - Utils.classForName(className).isAnnotationPresent(classOf[SQLUserDefinedType]) => - val udt = Utils.classForName(className) + case t if t.typeSymbol.annotations.exists(_.tpe =:= typeOf[SQLUserDefinedType]) => + val udt = getClassFromType(t) .getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance() val obj = NewInstance( udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt(), @@ -637,24 +633,6 @@ object ScalaReflection extends ScalaReflection { * Retrieves the runtime class corresponding to the provided type. */ def getClassFromType(tpe: Type): Class[_] = mirror.runtimeClass(tpe.erasure.typeSymbol.asClass) -} - -/** - * Support for generating catalyst schemas for scala objects. Note that unlike its companion - * object, this trait able to work in both the runtime and the compile time (macro) universe. - */ -trait ScalaReflection { - /** The universe we work in (runtime or macro) */ - val universe: scala.reflect.api.Universe - - /** The mirror used to access types in the universe */ - def mirror: universe.Mirror - - import universe._ - - // The Predef.Map is scala.collection.immutable.Map. - // Since the map values can be mutable, we explicitly import scala.collection.Map at here. - import scala.collection.Map case class Schema(dataType: DataType, nullable: Boolean) @@ -668,36 +646,22 @@ trait ScalaReflection { def schemaFor[T: TypeTag]: Schema = schemaFor(localTypeOf[T]) /** - * Return the Scala Type for `T` in the current classloader mirror. - * - * Use this method instead of the convenience method `universe.typeOf`, which - * assumes that all types can be found in the classloader that loaded scala-reflect classes. - * That's not necessarily the case when running using Eclipse launchers or even - * Sbt console or test (without `fork := true`). + * Returns a catalyst DataType and its nullability for the given Scala Type using reflection. * - * @see SPARK-5281 + * Unlike `schemaFor`, this method won't throw exception for un-supported type, it will return + * `NullType` silently instead. */ - // SPARK-13640: Synchronize this because TypeTag.tpe is not thread-safe in Scala 2.10. - def localTypeOf[T: TypeTag]: `Type` = ScalaReflectionLock.synchronized { - val tag = implicitly[TypeTag[T]] - tag.in(mirror).tpe.normalize + def silentSchemaFor(tpe: `Type`): Schema = try { + schemaFor(tpe) + } catch { + case _: UnsupportedOperationException => Schema(NullType, nullable = true) } /** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */ def schemaFor(tpe: `Type`): Schema = ScalaReflectionLock.synchronized { - val className = getClassNameFromType(tpe) - tpe match { - - case t if Utils.classIsLoadable(className) && - Utils.classForName(className).isAnnotationPresent(classOf[SQLUserDefinedType]) => - - // Note: We check for classIsLoadable above since Utils.classForName uses Java reflection, - // whereas className is from Scala reflection. This can make it hard to find classes - // in some cases, such as when a class is enclosed in an object (in which case - // Java appends a '$' to the object name but Scala does not). - val udt = Utils.classForName(className) - .getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance() + case t if t.typeSymbol.annotations.exists(_.tpe =:= typeOf[SQLUserDefinedType]) => + val udt = getClassFromType(t).getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance() Schema(udt, nullable = true) case t if t <:< localTypeOf[Option[_]] => val TypeRef(_, _, Seq(optType)) = t @@ -748,17 +712,39 @@ trait ScalaReflection { throw new UnsupportedOperationException(s"Schema for type $other is not supported") } } +} + +/** + * Support for generating catalyst schemas for scala objects. Note that unlike its companion + * object, this trait able to work in both the runtime and the compile time (macro) universe. + */ +trait ScalaReflection { + /** The universe we work in (runtime or macro) */ + val universe: scala.reflect.api.Universe + + /** The mirror used to access types in the universe */ + def mirror: universe.Mirror + + import universe._ + + // The Predef.Map is scala.collection.immutable.Map. + // Since the map values can be mutable, we explicitly import scala.collection.Map at here. + import scala.collection.Map /** - * Returns a catalyst DataType and its nullability for the given Scala Type using reflection. + * Return the Scala Type for `T` in the current classloader mirror. * - * Unlike `schemaFor`, this method won't throw exception for un-supported type, it will return - * `NullType` silently instead. + * Use this method instead of the convenience method `universe.typeOf`, which + * assumes that all types can be found in the classloader that loaded scala-reflect classes. + * That's not necessarily the case when running using Eclipse launchers or even + * Sbt console or test (without `fork := true`). + * + * @see SPARK-5281 */ - def silentSchemaFor(tpe: `Type`): Schema = try { - schemaFor(tpe) - } catch { - case _: UnsupportedOperationException => Schema(NullType, nullable = true) + // SPARK-13640: Synchronize this because TypeTag.tpe is not thread-safe in Scala 2.10. + def localTypeOf[T: TypeTag]: `Type` = ScalaReflectionLock.synchronized { + val tag = implicitly[TypeTag[T]] + tag.in(mirror).tpe.normalize } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index 5ca5a72512a29..0672551b2972d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -23,7 +23,7 @@ import java.sql.{Date, Timestamp} import scala.reflect.runtime.universe.typeOf import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.expressions.BoundReference +import org.apache.spark.sql.catalyst.expressions.{BoundReference, SpecificMutableRow} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -81,9 +81,44 @@ case class MultipleConstructorsData(a: Int, b: String, c: Double) { def this(b: String, a: Int) = this(a, b, c = 1.0) } +object TestingUDT { + @SQLUserDefinedType(udt = classOf[NestedStructUDT]) + class NestedStruct(val a: Integer, val b: Long, val c: Double) + + class NestedStructUDT extends UserDefinedType[NestedStruct] { + override def sqlType: DataType = new StructType() + .add("a", IntegerType, nullable = true) + .add("b", LongType, nullable = false) + .add("c", DoubleType, nullable = false) + + override def serialize(n: NestedStruct): Any = { + val row = new SpecificMutableRow(sqlType.asInstanceOf[StructType].map(_.dataType)) + row.setInt(0, n.a) + row.setLong(1, n.b) + row.setDouble(2, n.c) + } + + override def userClass: Class[NestedStruct] = classOf[NestedStruct] + + override def deserialize(datum: Any): NestedStruct = datum match { + case row: InternalRow => + new NestedStruct(row.getInt(0), row.getLong(1), row.getDouble(2)) + } + } +} + + class ScalaReflectionSuite extends SparkFunSuite { import org.apache.spark.sql.catalyst.ScalaReflection._ + test("SQLUserDefinedType annotation on Scala structure") { + val schema = schemaFor[TestingUDT.NestedStruct] + assert(schema === Schema( + new TestingUDT.NestedStructUDT, + nullable = true + )) + } + test("primitive data") { val schema = schemaFor[PrimitiveData] assert(schema === Schema( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index 8c4afb605b01f..acc9f48d7e08f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -27,51 +27,56 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ -@SQLUserDefinedType(udt = classOf[MyDenseVectorUDT]) -private[sql] class MyDenseVector(val data: Array[Double]) extends Serializable { - override def equals(other: Any): Boolean = other match { - case v: MyDenseVector => - java.util.Arrays.equals(this.data, v.data) - case _ => false - } -} - @BeanInfo private[sql] case class MyLabeledPoint( - @BeanProperty label: Double, - @BeanProperty features: MyDenseVector) + @BeanProperty label: Double, + @BeanProperty features: UDT.MyDenseVector) + +// Wrapped in an object to check Scala compatibility. See SPARK-13929 +object UDT { + + @SQLUserDefinedType(udt = classOf[MyDenseVectorUDT]) + private[sql] class MyDenseVector(val data: Array[Double]) extends Serializable { + override def equals(other: Any): Boolean = other match { + case v: MyDenseVector => + java.util.Arrays.equals(this.data, v.data) + case _ => false + } + } -private[sql] class MyDenseVectorUDT extends UserDefinedType[MyDenseVector] { + private[sql] class MyDenseVectorUDT extends UserDefinedType[MyDenseVector] { - override def sqlType: DataType = ArrayType(DoubleType, containsNull = false) + override def sqlType: DataType = ArrayType(DoubleType, containsNull = false) - override def serialize(features: MyDenseVector): ArrayData = { - new GenericArrayData(features.data.map(_.asInstanceOf[Any])) - } + override def serialize(features: MyDenseVector): ArrayData = { + new GenericArrayData(features.data.map(_.asInstanceOf[Any])) + } - override def deserialize(datum: Any): MyDenseVector = { - datum match { - case data: ArrayData => - new MyDenseVector(data.toDoubleArray()) + override def deserialize(datum: Any): MyDenseVector = { + datum match { + case data: ArrayData => + new MyDenseVector(data.toDoubleArray()) + } } - } - override def userClass: Class[MyDenseVector] = classOf[MyDenseVector] + override def userClass: Class[MyDenseVector] = classOf[MyDenseVector] - private[spark] override def asNullable: MyDenseVectorUDT = this + private[spark] override def asNullable: MyDenseVectorUDT = this - override def equals(other: Any): Boolean = other match { - case _: MyDenseVectorUDT => true - case _ => false + override def equals(other: Any): Boolean = other match { + case _: MyDenseVectorUDT => true + case _ => false + } } + } class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetTest { import testImplicits._ private lazy val pointsRDD = Seq( - MyLabeledPoint(1.0, new MyDenseVector(Array(0.1, 1.0))), - MyLabeledPoint(0.0, new MyDenseVector(Array(0.2, 2.0)))).toDF() + MyLabeledPoint(1.0, new UDT.MyDenseVector(Array(0.1, 1.0))), + MyLabeledPoint(0.0, new UDT.MyDenseVector(Array(0.2, 2.0)))).toDF() test("register user type: MyDenseVector for MyLabeledPoint") { val labels: RDD[Double] = pointsRDD.select('label).rdd.map { case Row(v: Double) => v } @@ -80,16 +85,16 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT assert(labelsArrays.contains(1.0)) assert(labelsArrays.contains(0.0)) - val features: RDD[MyDenseVector] = - pointsRDD.select('features).rdd.map { case Row(v: MyDenseVector) => v } - val featuresArrays: Array[MyDenseVector] = features.collect() + val features: RDD[UDT.MyDenseVector] = + pointsRDD.select('features).rdd.map { case Row(v: UDT.MyDenseVector) => v } + val featuresArrays: Array[UDT.MyDenseVector] = features.collect() assert(featuresArrays.size === 2) - assert(featuresArrays.contains(new MyDenseVector(Array(0.1, 1.0)))) - assert(featuresArrays.contains(new MyDenseVector(Array(0.2, 2.0)))) + assert(featuresArrays.contains(new UDT.MyDenseVector(Array(0.1, 1.0)))) + assert(featuresArrays.contains(new UDT.MyDenseVector(Array(0.2, 2.0)))) } test("UDTs and UDFs") { - sqlContext.udf.register("testType", (d: MyDenseVector) => d.isInstanceOf[MyDenseVector]) + sqlContext.udf.register("testType", (d: UDT.MyDenseVector) => d.isInstanceOf[UDT.MyDenseVector]) pointsRDD.registerTempTable("points") checkAnswer( sql("SELECT testType(features) from points"), @@ -103,8 +108,8 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT checkAnswer( sqlContext.read.parquet(path), Seq( - Row(1.0, new MyDenseVector(Array(0.1, 1.0))), - Row(0.0, new MyDenseVector(Array(0.2, 2.0))))) + Row(1.0, new UDT.MyDenseVector(Array(0.1, 1.0))), + Row(0.0, new UDT.MyDenseVector(Array(0.2, 2.0))))) } } @@ -115,18 +120,19 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT checkAnswer( sqlContext.read.parquet(path), Seq( - Row(1.0, new MyDenseVector(Array(0.1, 1.0))), - Row(0.0, new MyDenseVector(Array(0.2, 2.0))))) + Row(1.0, new UDT.MyDenseVector(Array(0.1, 1.0))), + Row(0.0, new UDT.MyDenseVector(Array(0.2, 2.0))))) } } // Tests to make sure that all operators correctly convert types on the way out. test("Local UDTs") { - val df = Seq((1, new MyDenseVector(Array(0.1, 1.0)))).toDF("int", "vec") - df.collect()(0).getAs[MyDenseVector](1) - df.take(1)(0).getAs[MyDenseVector](1) - df.limit(1).groupBy('int).agg(first('vec)).collect()(0).getAs[MyDenseVector](0) - df.orderBy('int).limit(1).groupBy('int).agg(first('vec)).collect()(0).getAs[MyDenseVector](0) + val df = Seq((1, new UDT.MyDenseVector(Array(0.1, 1.0)))).toDF("int", "vec") + df.collect()(0).getAs[UDT.MyDenseVector](1) + df.take(1)(0).getAs[UDT.MyDenseVector](1) + df.limit(1).groupBy('int).agg(first('vec)).collect()(0).getAs[UDT.MyDenseVector](0) + df.orderBy('int).limit(1).groupBy('int).agg(first('vec)).collect()(0) + .getAs[UDT.MyDenseVector](0) } test("UDTs with JSON") { @@ -136,26 +142,47 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT ) val schema = StructType(Seq( StructField("id", IntegerType, false), - StructField("vec", new MyDenseVectorUDT, false) + StructField("vec", new UDT.MyDenseVectorUDT, false) )) val stringRDD = sparkContext.parallelize(data) val jsonRDD = sqlContext.read.schema(schema).json(stringRDD) checkAnswer( jsonRDD, - Row(1, new MyDenseVector(Array(1.1, 2.2, 3.3, 4.4))) :: - Row(2, new MyDenseVector(Array(2.25, 4.5, 8.75))) :: + Row(1, new UDT.MyDenseVector(Array(1.1, 2.2, 3.3, 4.4))) :: + Row(2, new UDT.MyDenseVector(Array(2.25, 4.5, 8.75))) :: Nil ) } + test("UDTs with JSON and Dataset") { + val data = Seq( + "{\"id\":1,\"vec\":[1.1,2.2,3.3,4.4]}", + "{\"id\":2,\"vec\":[2.25,4.5,8.75]}" + ) + + val schema = StructType(Seq( + StructField("id", IntegerType, false), + StructField("vec", new UDT.MyDenseVectorUDT, false) + )) + + val stringRDD = sparkContext.parallelize(data) + val jsonDataset = sqlContext.read.schema(schema).json(stringRDD) + .as[(Int, UDT.MyDenseVector)] + checkDataset( + jsonDataset, + (1, new UDT.MyDenseVector(Array(1.1, 2.2, 3.3, 4.4))), + (2, new UDT.MyDenseVector(Array(2.25, 4.5, 8.75))) + ) + } + test("SPARK-10472 UserDefinedType.typeName") { assert(IntegerType.typeName === "integer") - assert(new MyDenseVectorUDT().typeName === "mydensevector") + assert(new UDT.MyDenseVectorUDT().typeName === "mydensevector") } test("Catalyst type converter null handling for UDTs") { - val udt = new MyDenseVectorUDT() + val udt = new UDT.MyDenseVectorUDT() val toScalaConverter = CatalystTypeConverters.createToScalaConverter(udt) assert(toScalaConverter(null) === null) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index e17340c70b7e6..1a7b62ca0ac77 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -1421,7 +1421,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { FloatType, DoubleType, DecimalType(25, 5), DecimalType(6, 5), DateType, TimestampType, ArrayType(IntegerType), MapType(StringType, LongType), struct, - new MyDenseVectorUDT()) + new UDT.MyDenseVectorUDT()) val fields = dataTypes.zipWithIndex.map { case (dataType, index) => StructField(s"col$index", dataType, nullable = true) } @@ -1445,7 +1445,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { Seq(2, 3, 4), Map("a string" -> 2000L), Row(4.75.toFloat, Seq(false, true)), - new MyDenseVector(Array(0.25, 2.25, 4.25))) + new UDT.MyDenseVector(Array(0.25, 2.25, 4.25))) val data = Row.fromSeq(Seq("Spark " + sqlContext.sparkContext.version) ++ constantValues) :: Nil diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala index 84bb7edf03821..bc87d3ef38ca2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala @@ -868,7 +868,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te FloatType, DoubleType, DecimalType(25, 5), DecimalType(6, 5), DateType, TimestampType, ArrayType(IntegerType), MapType(StringType, LongType), struct, - new MyDenseVectorUDT()) + new UDT.MyDenseVectorUDT()) // Right now, we will use SortBasedAggregate to handle UDAFs. // UnsafeRow.mutableFieldTypes.asScala.toSeq will trigger SortBasedAggregate to use // UnsafeRow as the aggregation buffer. While, dataTypes will trigger diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala index 3d0225679205a..368fe62ff2c71 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala @@ -113,7 +113,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes new StructType() .add("f1", FloatType, nullable = true) .add("f2", ArrayType(BooleanType, containsNull = true), nullable = true), - new MyDenseVectorUDT() + new UDT.MyDenseVectorUDT() ).filter(supportsDataType) try { From 4514aebd1e807a665c270bfdc3f1127b3a1da898 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Tue, 19 Apr 2016 19:48:03 -0700 Subject: [PATCH 43/69] [SPARK-14705][YARN] support Multiple FileSystem for YARN STAGING DIR ## What changes were proposed in this pull request? In SPARK-13063, It makes the SPARK YARN STAGING DIR as configurable. But it only support default FileSystem. If there are many clusters, It can be different FileSystem for different cluster in our spark. ## How was this patch tested? I have tested it successfully with following commands: MASTER=yarn-client ./bin/spark-shell --conf spark.yarn.stagingDir=hdfs:namenode2/temp $SPARK_HOME/bin/spark-submit --conf spark.yarn.stagingDir=hdfs:namenode2/temp cc tgravescs vanzin andrewor14 Author: Lianhui Wang Closes #12473 from lianhuiwang/SPARK-14705. --- .../org/apache/spark/deploy/yarn/Client.scala | 49 +++++++------------ .../spark/deploy/yarn/ClientSuite.scala | 12 ++--- 2 files changed, 25 insertions(+), 36 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 7c168ed279df6..ae5fb6bbd4f39 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -117,6 +117,11 @@ private[spark] class Client( private var appId: ApplicationId = null + // The app staging dir based on the STAGING_DIR configuration if configured + // otherwise based on the users home directory. + private val appStagingBaseDir = sparkConf.get(STAGING_DIR).map { new Path(_) } + .getOrElse(FileSystem.get(hadoopConf).getHomeDirectory()) + def reportLauncherState(state: SparkAppHandle.State): Unit = { launcherBackend.setState(state) } @@ -179,18 +184,17 @@ private[spark] class Client( * Cleanup application staging directory. */ private def cleanupStagingDir(appId: ApplicationId): Unit = { - val appStagingDir = getAppStagingDir(appId) + val stagingDirPath = new Path(appStagingBaseDir, getAppStagingDir(appId)) try { val preserveFiles = sparkConf.get(PRESERVE_STAGING_FILES) - val fs = FileSystem.get(hadoopConf) - val stagingDirPath = getAppStagingDirPath(sparkConf, fs, appStagingDir) + val fs = stagingDirPath.getFileSystem(hadoopConf) if (!preserveFiles && fs.exists(stagingDirPath)) { logInfo("Deleting staging directory " + stagingDirPath) fs.delete(stagingDirPath, true) } } catch { case ioe: IOException => - logWarning("Failed to cleanup staging dir " + appStagingDir, ioe) + logWarning("Failed to cleanup staging dir " + stagingDirPath, ioe) } } @@ -351,14 +355,13 @@ private[spark] class Client( * Exposed for testing. */ def prepareLocalResources( - appStagingDir: String, + destDir: Path, pySparkArchives: Seq[String]): HashMap[String, LocalResource] = { logInfo("Preparing resources for our AM container") // Upload Spark and the application JAR to the remote file system if necessary, // and add them as local resources to the application master. - val fs = FileSystem.get(hadoopConf) - val dst = getAppStagingDirPath(sparkConf, fs, appStagingDir) - val nns = YarnSparkHadoopUtil.get.getNameNodesToAccess(sparkConf) + dst + val fs = destDir.getFileSystem(hadoopConf) + val nns = YarnSparkHadoopUtil.get.getNameNodesToAccess(sparkConf) + destDir YarnSparkHadoopUtil.get.obtainTokensForNamenodes(nns, hadoopConf, credentials) // Used to keep track of URIs added to the distributed cache. If the same URI is added // multiple times, YARN will fail to launch containers for the app with an internal @@ -372,9 +375,9 @@ private[spark] class Client( YarnSparkHadoopUtil.get.obtainTokenForHBase(sparkConf, hadoopConf, credentials) val replication = sparkConf.get(STAGING_FILE_REPLICATION).map(_.toShort) - .getOrElse(fs.getDefaultReplication(dst)) + .getOrElse(fs.getDefaultReplication(destDir)) val localResources = HashMap[String, LocalResource]() - FileSystem.mkdirs(fs, dst, new FsPermission(STAGING_DIR_PERMISSION)) + FileSystem.mkdirs(fs, destDir, new FsPermission(STAGING_DIR_PERMISSION)) val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() @@ -422,7 +425,7 @@ private[spark] class Client( val localPath = getQualifiedLocalPath(localURI, hadoopConf) val linkname = targetDir.map(_ + "/").getOrElse("") + destName.orElse(Option(localURI.getFragment())).getOrElse(localPath.getName()) - val destPath = copyFileToRemote(dst, localPath, replication) + val destPath = copyFileToRemote(destDir, localPath, replication) val destFs = FileSystem.get(destPath.toUri(), hadoopConf) distCacheMgr.addResource( destFs, hadoopConf, destPath, localResources, resType, linkname, statCache, @@ -666,17 +669,15 @@ private[spark] class Client( * Set up the environment for launching our ApplicationMaster container. */ private def setupLaunchEnv( - stagingDir: String, + stagingDirPath: Path, pySparkArchives: Seq[String]): HashMap[String, String] = { logInfo("Setting up the launch environment for our AM container") val env = new HashMap[String, String]() populateClasspath(args, yarnConf, sparkConf, env, sparkConf.get(DRIVER_CLASS_PATH)) env("SPARK_YARN_MODE") = "true" - env("SPARK_YARN_STAGING_DIR") = stagingDir + env("SPARK_YARN_STAGING_DIR") = stagingDirPath.toString env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() if (loginFromKeytab) { - val remoteFs = FileSystem.get(hadoopConf) - val stagingDirPath = getAppStagingDirPath(sparkConf, remoteFs, stagingDir) val credentialsFile = "credentials-" + UUID.randomUUID().toString sparkConf.set(CREDENTIALS_FILE_PATH, new Path(stagingDirPath, credentialsFile).toString) logInfo(s"Credentials file set to: $credentialsFile") @@ -776,15 +777,15 @@ private[spark] class Client( : ContainerLaunchContext = { logInfo("Setting up container launch context for our AM") val appId = newAppResponse.getApplicationId - val appStagingDir = getAppStagingDir(appId) + val appStagingDirPath = new Path(appStagingBaseDir, getAppStagingDir(appId)) val pySparkArchives = if (sparkConf.get(IS_PYTHON_APP)) { findPySparkArchives() } else { Nil } - val launchEnv = setupLaunchEnv(appStagingDir, pySparkArchives) - val localResources = prepareLocalResources(appStagingDir, pySparkArchives) + val launchEnv = setupLaunchEnv(appStagingDirPath, pySparkArchives) + val localResources = prepareLocalResources(appStagingDirPath, pySparkArchives) // Set the environment variables to be passed on to the executors. distCacheMgr.setDistFilesEnv(launchEnv) @@ -1446,16 +1447,4 @@ private object Client extends Logging { uri.startsWith(s"$LOCAL_SCHEME:") } - /** - * Returns the app staging dir based on the STAGING_DIR configuration if configured - * otherwise based on the users home directory. - */ - private def getAppStagingDirPath( - conf: SparkConf, - fs: FileSystem, - appStagingDir: String): Path = { - val baseDir = conf.get(STAGING_DIR).map { new Path(_) }.getOrElse(fs.getHomeDirectory()) - new Path(baseDir, appStagingDir) - } - } diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index 23050e8c1d5c1..06efd44b5df9c 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -147,7 +147,7 @@ class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll val tempDir = Utils.createTempDir() try { - client.prepareLocalResources(tempDir.getAbsolutePath(), Nil) + client.prepareLocalResources(new Path(tempDir.getAbsolutePath()), Nil) sparkConf.get(APP_JAR) should be (Some(USER)) // The non-local path should be propagated by name only, since it will end up in the app's @@ -238,7 +238,7 @@ class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll val client = createClient(sparkConf) val tempDir = Utils.createTempDir() - client.prepareLocalResources(tempDir.getAbsolutePath(), Nil) + client.prepareLocalResources(new Path(tempDir.getAbsolutePath()), Nil) assert(sparkConf.get(SPARK_JARS) === Some(Seq(s"local:${jar4.getPath()}", s"local:${single.getAbsolutePath()}/*"))) @@ -260,14 +260,14 @@ class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll val sparkConf = new SparkConf().set(SPARK_ARCHIVE, archive.getPath()) val client = createClient(sparkConf) - client.prepareLocalResources(temp.getAbsolutePath(), Nil) + client.prepareLocalResources(new Path(temp.getAbsolutePath()), Nil) verify(client).copyFileToRemote(any(classOf[Path]), meq(new Path(archive.toURI())), anyShort()) classpath(client) should contain (buildPath(PWD, LOCALIZED_LIB_DIR, "*")) sparkConf.set(SPARK_ARCHIVE, LOCAL_SCHEME + ":" + archive.getPath()) intercept[IllegalArgumentException] { - client.prepareLocalResources(temp.getAbsolutePath(), Nil) + client.prepareLocalResources(new Path(temp.getAbsolutePath()), Nil) } } @@ -280,7 +280,7 @@ class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll val sparkConf = new SparkConfWithEnv(Map("SPARK_HOME" -> temp.getAbsolutePath())) val client = createClient(sparkConf) - client.prepareLocalResources(temp.getAbsolutePath(), Nil) + client.prepareLocalResources(new Path(temp.getAbsolutePath()), Nil) verify(client).copyFileToRemote(any(classOf[Path]), meq(new Path(jar.toURI())), anyShort()) classpath(client) should contain (buildPath(PWD, LOCALIZED_LIB_DIR, "*")) } @@ -308,7 +308,7 @@ class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll val client = createClient(sparkConf) val tempDir = Utils.createTempDir() - client.prepareLocalResources(tempDir.getAbsolutePath(), Nil) + client.prepareLocalResources(new Path(tempDir.getAbsolutePath()), Nil) // Only jar2 will be added to SECONDARY_JARS, jar3 which has the same name with jar1 will be // ignored. From 8eedf0b553180d0e958b0fb49bc2fee81658495c Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Tue, 19 Apr 2016 19:57:03 -0700 Subject: [PATCH 44/69] [SPARK-13905][SPARKR] Change signature of as.data.frame() to be consistent with the R base package. ## What changes were proposed in this pull request? Change the signature of as.data.frame() to be consistent with that in the R base package to meet R user's convention. ## How was this patch tested? dev/lint-r SparkR unit tests Author: Sun Rui Closes #11811 from sun-rui/SPARK-13905. --- R/pkg/R/DataFrame.R | 8 ++------ R/pkg/R/generics.R | 5 ++++- R/pkg/inst/tests/testthat/test_context.R | 2 +- R/pkg/inst/tests/testthat/test_sparkSQL.R | 3 +++ 4 files changed, 10 insertions(+), 8 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index ddb056fa71e7e..95e2eb2be037f 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -2296,12 +2296,8 @@ setMethod("fillna", #' } setMethod("as.data.frame", signature(x = "DataFrame"), - function(x, ...) { - # Check if additional parameters have been passed - if (length(list(...)) > 0) { - stop(paste("Unused argument(s): ", paste(list(...), collapse = ", "))) - } - collect(x) + function(x, row.names = NULL, optional = FALSE, ...) { + as.data.frame(collect(x), row.names, optional, ...) }) #' The specified DataFrame is attached to the R search path. This means that diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 4ef05d56bfb4f..a71be55bcae81 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -397,7 +397,10 @@ setGeneric("arrange", function(x, col, ...) { standardGeneric("arrange") }) #' @rdname as.data.frame #' @export -setGeneric("as.data.frame") +setGeneric("as.data.frame", + function(x, row.names = NULL, optional = FALSE, ...) { + standardGeneric("as.data.frame") + }) #' @rdname attach #' @export diff --git a/R/pkg/inst/tests/testthat/test_context.R b/R/pkg/inst/tests/testthat/test_context.R index 6e06c974c291f..9f51161230e1a 100644 --- a/R/pkg/inst/tests/testthat/test_context.R +++ b/R/pkg/inst/tests/testthat/test_context.R @@ -26,7 +26,7 @@ test_that("Check masked functions", { maskedBySparkR <- masked[funcSparkROrEmpty] namesOfMasked <- c("describe", "cov", "filter", "lag", "na.omit", "predict", "sd", "var", "colnames", "colnames<-", "intersect", "rank", "rbind", "sample", "subset", - "summary", "transform", "drop", "window") + "summary", "transform", "drop", "window", "as.data.frame") expect_equal(length(maskedBySparkR), length(namesOfMasked)) expect_equal(sort(maskedBySparkR), sort(namesOfMasked)) # above are those reported as masked when `library(SparkR)` diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index d747d4f83f24b..2f65484fcbdd8 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1863,6 +1863,9 @@ test_that("Method as.data.frame as a synonym for collect()", { expect_equal(as.data.frame(irisDF), collect(irisDF)) irisDF2 <- irisDF[irisDF$Species == "setosa", ] expect_equal(as.data.frame(irisDF2), collect(irisDF2)) + + # Make sure as.data.frame in the R base package is not covered + expect_that(as.data.frame(c(1, 2)), not(throws_error())) }) test_that("attach() on a DataFrame", { From 78b38109ed2fc20e97f9a968185d0c02ef83aa42 Mon Sep 17 00:00:00 2001 From: Luciano Resende Date: Tue, 19 Apr 2016 21:02:10 -0700 Subject: [PATCH 45/69] [SPARK-13419] [SQL] Update SubquerySuite to use checkAnswer for validation ## What changes were proposed in this pull request? Change SubquerySuite to validate test results utilizing checkAnswer helper method ## How was this patch tested? Existing tests Author: Luciano Resende Closes #12269 from lresende/SPARK-13419. --- .../org/apache/spark/sql/SubquerySuite.scala | 66 +++++++++++-------- 1 file changed, 38 insertions(+), 28 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 5742983fb9d07..d69ef087357d8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -55,32 +55,37 @@ class SubquerySuite extends QueryTest with SharedSQLContext { } test("simple uncorrelated scalar subquery") { - assertResult(Array(Row(1))) { - sql("select (select 1 as b) as b").collect() - } + checkAnswer( + sql("select (select 1 as b) as b"), + Array(Row(1)) + ) - assertResult(Array(Row(3))) { - sql("select (select (select 1) + 1) + 1").collect() - } + checkAnswer( + sql("select (select (select 1) + 1) + 1"), + Array(Row(3)) + ) // string type - assertResult(Array(Row("s"))) { - sql("select (select 's' as s) as b").collect() - } + checkAnswer( + sql("select (select 's' as s) as b"), + Array(Row("s")) + ) } test("uncorrelated scalar subquery in CTE") { - assertResult(Array(Row(1))) { + checkAnswer( sql("with t2 as (select 1 as b, 2 as c) " + "select a from (select 1 as a union all select 2 as a) t " + - "where a = (select max(b) from t2) ").collect() - } + "where a = (select max(b) from t2) "), + Array(Row(1)) + ) } test("uncorrelated scalar subquery should return null if there is 0 rows") { - assertResult(Array(Row(null))) { - sql("select (select 's' as s limit 0) as b").collect() - } + checkAnswer( + sql("select (select 's' as s limit 0) as b"), + Array(Row(null)) + ) } test("runtime error when the number of rows is greater than 1") { @@ -88,29 +93,34 @@ class SubquerySuite extends QueryTest with SharedSQLContext { sql("select (select a from (select 1 as a union all select 2 as a) t) as b").collect() } assert(error2.getMessage.contains( - "more than one row returned by a subquery used as an expression")) + "more than one row returned by a subquery used as an expression") + ) } test("uncorrelated scalar subquery on a DataFrame generated query") { val df = Seq((1, "one"), (2, "two"), (3, "three")).toDF("key", "value") df.registerTempTable("subqueryData") - assertResult(Array(Row(4))) { - sql("select (select key from subqueryData where key > 2 order by key limit 1) + 1").collect() - } + checkAnswer( + sql("select (select key from subqueryData where key > 2 order by key limit 1) + 1"), + Array(Row(4)) + ) - assertResult(Array(Row(-3))) { - sql("select -(select max(key) from subqueryData)").collect() - } + checkAnswer( + sql("select -(select max(key) from subqueryData)"), + Array(Row(-3)) + ) - assertResult(Array(Row(null))) { - sql("select (select value from subqueryData limit 0)").collect() - } + checkAnswer( + sql("select (select value from subqueryData limit 0)"), + Array(Row(null)) + ) - assertResult(Array(Row("two"))) { + checkAnswer( sql("select (select min(value) from subqueryData" + - " where key = (select max(key) from subqueryData) - 1)").collect() - } + " where key = (select max(key) from subqueryData) - 1)"), + Array(Row("two")) + ) } test("EXISTS predicate subquery") { From 85d759ca3aebb7d60b963207dcada83c75502e52 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 19 Apr 2016 21:20:24 -0700 Subject: [PATCH 46/69] [SPARK-14704][CORE] create accumulators in TaskMetrics ## What changes were proposed in this pull request? Before this PR, we create accumulators at driver side(and register them) and send them to executor side, then we create `TaskMetrics` with these accumulators at executor side. After this PR, we will create `TaskMetrics` at driver side and send it to executor side, so that we can create accumulators inside `TaskMetrics` directly, which is cleaner. ## How was this patch tested? existing tests. Author: Wenchen Fan Closes #12472 from cloud-fan/acc. --- .../apache/spark/InternalAccumulator.scala | 127 -------- .../scala/org/apache/spark/TaskContext.scala | 3 +- .../org/apache/spark/TaskContextImpl.scala | 7 +- .../org/apache/spark/executor/Executor.scala | 32 +- .../apache/spark/executor/InputMetrics.scala | 14 +- .../apache/spark/executor/OutputMetrics.scala | 15 +- .../spark/executor/ShuffleReadMetrics.scala | 49 +--- .../spark/executor/ShuffleWriteMetrics.scala | 34 +-- .../apache/spark/executor/TaskMetrics.scala | 175 +++++------ .../apache/spark/scheduler/DAGScheduler.scala | 4 +- .../apache/spark/scheduler/ResultTask.scala | 10 +- .../spark/scheduler/ShuffleMapTask.scala | 10 +- .../org/apache/spark/scheduler/Stage.scala | 6 +- .../apache/spark/scheduler/StageInfo.scala | 8 +- .../org/apache/spark/scheduler/Task.scala | 16 +- .../org/apache/spark/ui/jobs/JobPage.scala | 2 +- .../org/apache/spark/util/JsonProtocol.scala | 41 ++- .../org/apache/spark/AccumulatorSuite.scala | 18 +- .../ExecutorAllocationManagerSuite.scala | 2 +- .../spark/InternalAccumulatorSuite.scala | 129 +------- .../scala/org/apache/spark/ShuffleSuite.scala | 9 +- .../spark/executor/TaskMetricsSuite.scala | 277 ++++-------------- .../spark/scheduler/DAGSchedulerSuite.scala | 4 +- .../org/apache/spark/scheduler/FakeTask.scala | 5 +- .../scheduler/NotSerializableFakeTask.scala | 3 +- .../spark/scheduler/TaskContextSuite.scala | 21 +- .../spark/scheduler/TaskSetManagerSuite.scala | 11 +- .../api/v1/AllStagesResourceSuite.scala | 2 +- .../ui/jobs/JobProgressListenerSuite.scala | 4 +- project/MimaExcludes.scala | 4 + .../execution/UnsafeRowSerializerSuite.scala | 4 +- 31 files changed, 271 insertions(+), 775 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/InternalAccumulator.scala b/core/src/main/scala/org/apache/spark/InternalAccumulator.scala index 714c8737a9b62..0b494c146fa1b 100644 --- a/core/src/main/scala/org/apache/spark/InternalAccumulator.scala +++ b/core/src/main/scala/org/apache/spark/InternalAccumulator.scala @@ -17,17 +17,11 @@ package org.apache.spark -import org.apache.spark.storage.{BlockId, BlockStatus} - - /** * A collection of fields and methods concerned with internal accumulators that represent * task level metrics. */ private[spark] object InternalAccumulator { - - import AccumulatorParam._ - // Prefixes used in names of internal task level metrics val METRICS_PREFIX = "internal.metrics." val SHUFFLE_READ_METRICS_PREFIX = METRICS_PREFIX + "shuffle.read." @@ -79,125 +73,4 @@ private[spark] object InternalAccumulator { } // scalastyle:on - - /** - * Create an internal [[Accumulator]] by name, which must begin with [[METRICS_PREFIX]]. - */ - def create(name: String): Accumulator[_] = { - require(name.startsWith(METRICS_PREFIX), - s"internal accumulator name must start with '$METRICS_PREFIX': $name") - getParam(name) match { - case p @ LongAccumulatorParam => newMetric[Long](0L, name, p) - case p @ IntAccumulatorParam => newMetric[Int](0, name, p) - case p @ StringAccumulatorParam => newMetric[String]("", name, p) - case p @ UpdatedBlockStatusesAccumulatorParam => - newMetric[Seq[(BlockId, BlockStatus)]](Seq(), name, p) - case p => throw new IllegalArgumentException( - s"unsupported accumulator param '${p.getClass.getSimpleName}' for metric '$name'.") - } - } - - /** - * Get the [[AccumulatorParam]] associated with the internal metric name, - * which must begin with [[METRICS_PREFIX]]. - */ - def getParam(name: String): AccumulatorParam[_] = { - require(name.startsWith(METRICS_PREFIX), - s"internal accumulator name must start with '$METRICS_PREFIX': $name") - name match { - case UPDATED_BLOCK_STATUSES => UpdatedBlockStatusesAccumulatorParam - case shuffleRead.LOCAL_BLOCKS_FETCHED => IntAccumulatorParam - case shuffleRead.REMOTE_BLOCKS_FETCHED => IntAccumulatorParam - case _ => LongAccumulatorParam - } - } - - /** - * Accumulators for tracking internal metrics. - */ - def createAll(): Seq[Accumulator[_]] = { - Seq[String]( - EXECUTOR_DESERIALIZE_TIME, - EXECUTOR_RUN_TIME, - RESULT_SIZE, - JVM_GC_TIME, - RESULT_SERIALIZATION_TIME, - MEMORY_BYTES_SPILLED, - DISK_BYTES_SPILLED, - PEAK_EXECUTION_MEMORY, - UPDATED_BLOCK_STATUSES).map(create) ++ - createShuffleReadAccums() ++ - createShuffleWriteAccums() ++ - createInputAccums() ++ - createOutputAccums() ++ - sys.props.get("spark.testing").map(_ => create(TEST_ACCUM)).toSeq - } - - /** - * Accumulators for tracking shuffle read metrics. - */ - def createShuffleReadAccums(): Seq[Accumulator[_]] = { - Seq[String]( - shuffleRead.REMOTE_BLOCKS_FETCHED, - shuffleRead.LOCAL_BLOCKS_FETCHED, - shuffleRead.REMOTE_BYTES_READ, - shuffleRead.LOCAL_BYTES_READ, - shuffleRead.FETCH_WAIT_TIME, - shuffleRead.RECORDS_READ).map(create) - } - - /** - * Accumulators for tracking shuffle write metrics. - */ - def createShuffleWriteAccums(): Seq[Accumulator[_]] = { - Seq[String]( - shuffleWrite.BYTES_WRITTEN, - shuffleWrite.RECORDS_WRITTEN, - shuffleWrite.WRITE_TIME).map(create) - } - - /** - * Accumulators for tracking input metrics. - */ - def createInputAccums(): Seq[Accumulator[_]] = { - Seq[String]( - input.BYTES_READ, - input.RECORDS_READ).map(create) - } - - /** - * Accumulators for tracking output metrics. - */ - def createOutputAccums(): Seq[Accumulator[_]] = { - Seq[String]( - output.BYTES_WRITTEN, - output.RECORDS_WRITTEN).map(create) - } - - /** - * Accumulators for tracking internal metrics. - * - * These accumulators are created with the stage such that all tasks in the stage will - * add to the same set of accumulators. We do this to report the distribution of accumulator - * values across all tasks within each stage. - */ - def createAll(sc: SparkContext): Seq[Accumulator[_]] = { - val accums = createAll() - accums.foreach { accum => - Accumulators.register(accum) - sc.cleaner.foreach(_.registerAccumulatorForCleanup(accum)) - } - accums - } - - /** - * Create a new accumulator representing an internal task metric. - */ - private def newMetric[T]( - initialValue: T, - name: String, - param: AccumulatorParam[T]): Accumulator[T] = { - new Accumulator[T](initialValue, param, Some(name), internal = true, countFailedValues = true) - } - } diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 757c1b5116f3c..e7940bd9eddcd 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -62,12 +62,11 @@ object TaskContext { protected[spark] def unset(): Unit = taskContext.remove() /** - * An empty task context that does not represent an actual task. + * An empty task context that does not represent an actual task. This is only used in tests. */ private[spark] def empty(): TaskContextImpl = { new TaskContextImpl(0, 0, 0, 0, null, new Properties, null) } - } diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index fa0b2d3d28293..e8f83c6d14b37 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -36,15 +36,10 @@ private[spark] class TaskContextImpl( override val taskMemoryManager: TaskMemoryManager, localProperties: Properties, @transient private val metricsSystem: MetricsSystem, - initialAccumulators: Seq[Accumulator[_]] = InternalAccumulator.createAll()) + override val taskMetrics: TaskMetrics = new TaskMetrics) extends TaskContext with Logging { - /** - * Metrics associated with this task. - */ - override val taskMetrics: TaskMetrics = new TaskMetrics(initialAccumulators) - /** List of callback functions to execute when the task completes. */ @transient private val onCompleteCallbacks = new ArrayBuffer[TaskCompletionListener] diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index b20bd11f7d9ec..650f05c309d20 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -293,16 +293,14 @@ private[spark] class Executor( val valueBytes = resultSer.serialize(value) val afterSerialization = System.currentTimeMillis() - for (m <- task.metrics) { - // Deserialization happens in two parts: first, we deserialize a Task object, which - // includes the Partition. Second, Task.run() deserializes the RDD and function to be run. - m.setExecutorDeserializeTime( - (taskStart - deserializeStartTime) + task.executorDeserializeTime) - // We need to subtract Task.run()'s deserialization time to avoid double-counting - m.setExecutorRunTime((taskFinish - taskStart) - task.executorDeserializeTime) - m.setJvmGCTime(computeTotalGcTime() - startGCTime) - m.setResultSerializationTime(afterSerialization - beforeSerialization) - } + // Deserialization happens in two parts: first, we deserialize a Task object, which + // includes the Partition. Second, Task.run() deserializes the RDD and function to be run. + task.metrics.setExecutorDeserializeTime( + (taskStart - deserializeStartTime) + task.executorDeserializeTime) + // We need to subtract Task.run()'s deserialization time to avoid double-counting + task.metrics.setExecutorRunTime((taskFinish - taskStart) - task.executorDeserializeTime) + task.metrics.setJvmGCTime(computeTotalGcTime() - startGCTime) + task.metrics.setResultSerializationTime(afterSerialization - beforeSerialization) // Note: accumulator updates must be collected after TaskMetrics is updated val accumUpdates = task.collectAccumulatorUpdates() @@ -357,10 +355,8 @@ private[spark] class Executor( // Collect latest accumulator values to report back to the driver val accumulatorUpdates: Seq[AccumulableInfo] = if (task != null) { - task.metrics.foreach { m => - m.setExecutorRunTime(System.currentTimeMillis() - taskStart) - m.setJvmGCTime(computeTotalGcTime() - startGCTime) - } + task.metrics.setExecutorRunTime(System.currentTimeMillis() - taskStart) + task.metrics.setJvmGCTime(computeTotalGcTime() - startGCTime) task.collectAccumulatorUpdates(taskFailed = true) } else { Seq.empty[AccumulableInfo] @@ -485,11 +481,9 @@ private[spark] class Executor( for (taskRunner <- runningTasks.values().asScala) { if (taskRunner.task != null) { - taskRunner.task.metrics.foreach { metrics => - metrics.mergeShuffleReadMetrics() - metrics.setJvmGCTime(curGCTime - taskRunner.startGCTime) - accumUpdates += ((taskRunner.taskId, metrics.accumulatorUpdates())) - } + taskRunner.task.metrics.mergeShuffleReadMetrics() + taskRunner.task.metrics.setJvmGCTime(curGCTime - taskRunner.startGCTime) + accumUpdates += ((taskRunner.taskId, taskRunner.task.metrics.accumulatorUpdates())) } } diff --git a/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala index 0ec81d8d3588a..535352e7dd7a1 100644 --- a/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/InputMetrics.scala @@ -17,7 +17,7 @@ package org.apache.spark.executor -import org.apache.spark.{Accumulator, InternalAccumulator} +import org.apache.spark.InternalAccumulator import org.apache.spark.annotation.DeveloperApi @@ -39,14 +39,11 @@ object DataReadMethod extends Enumeration with Serializable { * A collection of accumulators that represents metrics about reading data from external systems. */ @DeveloperApi -class InputMetrics private (_bytesRead: Accumulator[Long], _recordsRead: Accumulator[Long]) - extends Serializable { +class InputMetrics private[spark] () extends Serializable { + import InternalAccumulator._ - private[executor] def this(accumMap: Map[String, Accumulator[_]]) { - this( - TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.input.BYTES_READ), - TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.input.RECORDS_READ)) - } + private[executor] val _bytesRead = TaskMetrics.createLongAccum(input.BYTES_READ) + private[executor] val _recordsRead = TaskMetrics.createLongAccum(input.RECORDS_READ) /** * Total number of bytes read. @@ -61,5 +58,4 @@ class InputMetrics private (_bytesRead: Accumulator[Long], _recordsRead: Accumul private[spark] def incBytesRead(v: Long): Unit = _bytesRead.add(v) private[spark] def incRecordsRead(v: Long): Unit = _recordsRead.add(v) private[spark] def setBytesRead(v: Long): Unit = _bytesRead.setValue(v) - } diff --git a/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala index 5b36cc4739373..586c98b15637b 100644 --- a/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/OutputMetrics.scala @@ -17,7 +17,7 @@ package org.apache.spark.executor -import org.apache.spark.{Accumulator, InternalAccumulator} +import org.apache.spark.InternalAccumulator import org.apache.spark.annotation.DeveloperApi @@ -38,14 +38,11 @@ object DataWriteMethod extends Enumeration with Serializable { * A collection of accumulators that represents metrics about writing data to external systems. */ @DeveloperApi -class OutputMetrics private (_bytesWritten: Accumulator[Long], _recordsWritten: Accumulator[Long]) - extends Serializable { - - private[executor] def this(accumMap: Map[String, Accumulator[_]]) { - this( - TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.output.BYTES_WRITTEN), - TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.output.RECORDS_WRITTEN)) - } +class OutputMetrics private[spark] () extends Serializable { + import InternalAccumulator._ + + private[executor] val _bytesWritten = TaskMetrics.createLongAccum(output.BYTES_WRITTEN) + private[executor] val _recordsWritten = TaskMetrics.createLongAccum(output.RECORDS_WRITTEN) /** * Total number of bytes written. diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala index 47cfb74b9eb7d..8e9a332b7c556 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala @@ -17,7 +17,7 @@ package org.apache.spark.executor -import org.apache.spark.{Accumulator, InternalAccumulator} +import org.apache.spark.InternalAccumulator import org.apache.spark.annotation.DeveloperApi @@ -27,38 +27,21 @@ import org.apache.spark.annotation.DeveloperApi * Operations are not thread-safe. */ @DeveloperApi -class ShuffleReadMetrics private ( - _remoteBlocksFetched: Accumulator[Int], - _localBlocksFetched: Accumulator[Int], - _remoteBytesRead: Accumulator[Long], - _localBytesRead: Accumulator[Long], - _fetchWaitTime: Accumulator[Long], - _recordsRead: Accumulator[Long]) - extends Serializable { - - private[executor] def this(accumMap: Map[String, Accumulator[_]]) { - this( - TaskMetrics.getAccum[Int](accumMap, InternalAccumulator.shuffleRead.REMOTE_BLOCKS_FETCHED), - TaskMetrics.getAccum[Int](accumMap, InternalAccumulator.shuffleRead.LOCAL_BLOCKS_FETCHED), - TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleRead.REMOTE_BYTES_READ), - TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleRead.LOCAL_BYTES_READ), - TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleRead.FETCH_WAIT_TIME), - TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleRead.RECORDS_READ)) - } - - /** - * Create a new [[ShuffleReadMetrics]] that is not associated with any particular task. - * - * This mainly exists for legacy reasons, because we use dummy [[ShuffleReadMetrics]] in - * many places only to merge their values together later. In the future, we should revisit - * whether this is needed. - * - * A better alternative is [[TaskMetrics.createTempShuffleReadMetrics]] followed by - * [[TaskMetrics.mergeShuffleReadMetrics]]. - */ - private[spark] def this() { - this(InternalAccumulator.createShuffleReadAccums().map { a => (a.name.get, a) }.toMap) - } +class ShuffleReadMetrics private[spark] () extends Serializable { + import InternalAccumulator._ + + private[executor] val _remoteBlocksFetched = + TaskMetrics.createIntAccum(shuffleRead.REMOTE_BLOCKS_FETCHED) + private[executor] val _localBlocksFetched = + TaskMetrics.createIntAccum(shuffleRead.LOCAL_BLOCKS_FETCHED) + private[executor] val _remoteBytesRead = + TaskMetrics.createLongAccum(shuffleRead.REMOTE_BYTES_READ) + private[executor] val _localBytesRead = + TaskMetrics.createLongAccum(shuffleRead.LOCAL_BYTES_READ) + private[executor] val _fetchWaitTime = + TaskMetrics.createLongAccum(shuffleRead.FETCH_WAIT_TIME) + private[executor] val _recordsRead = + TaskMetrics.createLongAccum(shuffleRead.RECORDS_READ) /** * Number of remote blocks fetched in this shuffle by this task. diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala index 704dee747eda1..7326fba841587 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala @@ -17,7 +17,7 @@ package org.apache.spark.executor -import org.apache.spark.{Accumulator, InternalAccumulator} +import org.apache.spark.InternalAccumulator import org.apache.spark.annotation.DeveloperApi @@ -27,31 +27,15 @@ import org.apache.spark.annotation.DeveloperApi * Operations are not thread-safe. */ @DeveloperApi -class ShuffleWriteMetrics private ( - _bytesWritten: Accumulator[Long], - _recordsWritten: Accumulator[Long], - _writeTime: Accumulator[Long]) - extends Serializable { +class ShuffleWriteMetrics private[spark] () extends Serializable { + import InternalAccumulator._ - private[executor] def this(accumMap: Map[String, Accumulator[_]]) { - this( - TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleWrite.BYTES_WRITTEN), - TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleWrite.RECORDS_WRITTEN), - TaskMetrics.getAccum[Long](accumMap, InternalAccumulator.shuffleWrite.WRITE_TIME)) - } - - /** - * Create a new [[ShuffleWriteMetrics]] that is not associated with any particular task. - * - * This mainly exists for legacy reasons, because we use dummy [[ShuffleWriteMetrics]] in - * many places only to merge their values together later. In the future, we should revisit - * whether this is needed. - * - * A better alternative is [[TaskMetrics.shuffleWriteMetrics]]. - */ - private[spark] def this() { - this(InternalAccumulator.createShuffleWriteAccums().map { a => (a.name.get, a) }.toMap) - } + private[executor] val _bytesWritten = + TaskMetrics.createLongAccum(shuffleWrite.BYTES_WRITTEN) + private[executor] val _recordsWritten = + TaskMetrics.createLongAccum(shuffleWrite.RECORDS_WRITTEN) + private[executor] val _writeTime = + TaskMetrics.createLongAccum(shuffleWrite.WRITE_TIME) /** * Number of bytes written for the shuffle by this task. diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 01983648252d4..4558fbb4d95d8 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,10 +17,10 @@ package org.apache.spark.executor -import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import org.apache.spark._ +import org.apache.spark.AccumulatorParam.{IntAccumulatorParam, LongAccumulatorParam, UpdatedBlockStatusesAccumulatorParam} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging import org.apache.spark.scheduler.AccumulableInfo @@ -39,65 +39,21 @@ import org.apache.spark.storage.{BlockId, BlockStatus} * The accumulator updates are also sent to the driver periodically (on executor heartbeat) * and when the task failed with an exception. The [[TaskMetrics]] object itself should never * be sent to the driver. - * - * @param initialAccums the initial set of accumulators that this [[TaskMetrics]] depends on. - * Each accumulator in this initial set must be uniquely named and marked - * as internal. Additional accumulators registered later need not satisfy - * these requirements. */ @DeveloperApi -class TaskMetrics private[spark] (initialAccums: Seq[Accumulator[_]]) extends Serializable { +class TaskMetrics private[spark] () extends Serializable { import InternalAccumulator._ - // Needed for Java tests - def this() { - this(InternalAccumulator.createAll()) - } - - /** - * All accumulators registered with this task. - */ - private val accums = new ArrayBuffer[Accumulable[_, _]] - accums ++= initialAccums - - /** - * A map for quickly accessing the initial set of accumulators by name. - */ - private val initialAccumsMap: Map[String, Accumulator[_]] = { - val map = new mutable.HashMap[String, Accumulator[_]] - initialAccums.foreach { a => - val name = a.name.getOrElse { - throw new IllegalArgumentException( - "initial accumulators passed to TaskMetrics must be named") - } - require(a.isInternal, - s"initial accumulator '$name' passed to TaskMetrics must be marked as internal") - require(!map.contains(name), - s"detected duplicate accumulator name '$name' when constructing TaskMetrics") - map(name) = a - } - map.toMap - } - // Each metric is internally represented as an accumulator - private val _executorDeserializeTime = getAccum(EXECUTOR_DESERIALIZE_TIME) - private val _executorRunTime = getAccum(EXECUTOR_RUN_TIME) - private val _resultSize = getAccum(RESULT_SIZE) - private val _jvmGCTime = getAccum(JVM_GC_TIME) - private val _resultSerializationTime = getAccum(RESULT_SERIALIZATION_TIME) - private val _memoryBytesSpilled = getAccum(MEMORY_BYTES_SPILLED) - private val _diskBytesSpilled = getAccum(DISK_BYTES_SPILLED) - private val _peakExecutionMemory = getAccum(PEAK_EXECUTION_MEMORY) - private val _updatedBlockStatuses = - TaskMetrics.getAccum[Seq[(BlockId, BlockStatus)]](initialAccumsMap, UPDATED_BLOCK_STATUSES) - - private val _inputMetrics = new InputMetrics(initialAccumsMap) - - private val _outputMetrics = new OutputMetrics(initialAccumsMap) - - private val _shuffleReadMetrics = new ShuffleReadMetrics(initialAccumsMap) - - private val _shuffleWriteMetrics = new ShuffleWriteMetrics(initialAccumsMap) + private val _executorDeserializeTime = TaskMetrics.createLongAccum(EXECUTOR_DESERIALIZE_TIME) + private val _executorRunTime = TaskMetrics.createLongAccum(EXECUTOR_RUN_TIME) + private val _resultSize = TaskMetrics.createLongAccum(RESULT_SIZE) + private val _jvmGCTime = TaskMetrics.createLongAccum(JVM_GC_TIME) + private val _resultSerializationTime = TaskMetrics.createLongAccum(RESULT_SERIALIZATION_TIME) + private val _memoryBytesSpilled = TaskMetrics.createLongAccum(MEMORY_BYTES_SPILLED) + private val _diskBytesSpilled = TaskMetrics.createLongAccum(DISK_BYTES_SPILLED) + private val _peakExecutionMemory = TaskMetrics.createLongAccum(PEAK_EXECUTION_MEMORY) + private val _updatedBlockStatuses = TaskMetrics.createBlocksAccum(UPDATED_BLOCK_STATUSES) /** * Time taken on the executor to deserialize this task. @@ -163,31 +119,28 @@ class TaskMetrics private[spark] (initialAccums: Seq[Accumulator[_]]) extends Se private[spark] def setUpdatedBlockStatuses(v: Seq[(BlockId, BlockStatus)]): Unit = _updatedBlockStatuses.setValue(v) - /** - * Get a Long accumulator from the given map by name, assuming it exists. - * Note: this only searches the initial set of accumulators passed into the constructor. - */ - private[spark] def getAccum(name: String): Accumulator[Long] = { - TaskMetrics.getAccum[Long](initialAccumsMap, name) - } - /** * Metrics related to reading data from a [[org.apache.spark.rdd.HadoopRDD]] or from persisted * data, defined only in tasks with input. */ - def inputMetrics: InputMetrics = _inputMetrics + val inputMetrics: InputMetrics = new InputMetrics() /** * Metrics related to writing data externally (e.g. to a distributed filesystem), * defined only in tasks with output. */ - def outputMetrics: OutputMetrics = _outputMetrics + val outputMetrics: OutputMetrics = new OutputMetrics() /** * Metrics related to shuffle read aggregated across all shuffle dependencies. * This is defined only if there are shuffle dependencies in this task. */ - def shuffleReadMetrics: ShuffleReadMetrics = _shuffleReadMetrics + val shuffleReadMetrics: ShuffleReadMetrics = new ShuffleReadMetrics() + + /** + * Metrics related to shuffle write, defined only in shuffle map stages. + */ + val shuffleWriteMetrics: ShuffleWriteMetrics = new ShuffleWriteMetrics() /** * Temporary list of [[ShuffleReadMetrics]], one per shuffle dependency. @@ -217,21 +170,45 @@ class TaskMetrics private[spark] (initialAccums: Seq[Accumulator[_]]) extends Se */ private[spark] def mergeShuffleReadMetrics(): Unit = synchronized { if (tempShuffleReadMetrics.nonEmpty) { - _shuffleReadMetrics.setMergeValues(tempShuffleReadMetrics) + shuffleReadMetrics.setMergeValues(tempShuffleReadMetrics) } } - /** - * Metrics related to shuffle write, defined only in shuffle map stages. - */ - def shuffleWriteMetrics: ShuffleWriteMetrics = _shuffleWriteMetrics + // Only used for test + private[spark] val testAccum = + sys.props.get("spark.testing").map(_ => TaskMetrics.createLongAccum(TEST_ACCUM)) + + @transient private[spark] lazy val internalAccums: Seq[Accumulable[_, _]] = { + val in = inputMetrics + val out = outputMetrics + val sr = shuffleReadMetrics + val sw = shuffleWriteMetrics + Seq(_executorDeserializeTime, _executorRunTime, _resultSize, _jvmGCTime, + _resultSerializationTime, _memoryBytesSpilled, _diskBytesSpilled, _peakExecutionMemory, + _updatedBlockStatuses, sr._remoteBlocksFetched, sr._localBlocksFetched, sr._remoteBytesRead, + sr._localBytesRead, sr._fetchWaitTime, sr._recordsRead, sw._bytesWritten, sw._recordsWritten, + sw._writeTime, in._bytesRead, in._recordsRead, out._bytesWritten, out._recordsWritten) ++ + testAccum + } /* ========================== * | OTHER THINGS | * ========================== */ + private[spark] def registerAccums(sc: SparkContext): Unit = { + internalAccums.foreach { accum => + Accumulators.register(accum) + sc.cleaner.foreach(_.registerAccumulatorForCleanup(accum)) + } + } + + /** + * External accumulators registered with this task. + */ + @transient private lazy val externalAccums = new ArrayBuffer[Accumulable[_, _]] + private[spark] def registerAccumulator(a: Accumulable[_, _]): Unit = { - accums += a + externalAccums += a } /** @@ -242,7 +219,7 @@ class TaskMetrics private[spark] (initialAccums: Seq[Accumulator[_]]) extends Se * not the aggregated value across multiple tasks. */ def accumulatorUpdates(): Seq[AccumulableInfo] = { - accums.map { a => a.toInfo(Some(a.localValue), None) } + (internalAccums ++ externalAccums).map { a => a.toInfo(Some(a.localValue), None) } } } @@ -256,9 +233,7 @@ class TaskMetrics private[spark] (initialAccums: Seq[Accumulator[_]]) extends Se * UnsupportedOperationException, we choose not to do so because the overrides would quickly become * out-of-date when new metrics are added. */ -private[spark] class ListenerTaskMetrics( - initialAccums: Seq[Accumulator[_]], - accumUpdates: Seq[AccumulableInfo]) extends TaskMetrics(initialAccums) { +private[spark] class ListenerTaskMetrics(accumUpdates: Seq[AccumulableInfo]) extends TaskMetrics { override def accumulatorUpdates(): Seq[AccumulableInfo] = accumUpdates @@ -272,18 +247,25 @@ private[spark] object TaskMetrics extends Logging { def empty: TaskMetrics = new TaskMetrics /** - * Get an accumulator from the given map by name, assuming it exists. + * Create a new accumulator representing an internal task metric. */ - def getAccum[T](accumMap: Map[String, Accumulator[_]], name: String): Accumulator[T] = { - require(accumMap.contains(name), s"metric '$name' is missing") - val accum = accumMap(name) - try { - // Note: we can't do pattern matching here because types are erased by compile time - accum.asInstanceOf[Accumulator[T]] - } catch { - case e: ClassCastException => - throw new SparkException(s"accumulator $name was of unexpected type", e) - } + private def newMetric[T]( + initialValue: T, + name: String, + param: AccumulatorParam[T]): Accumulator[T] = { + new Accumulator[T](initialValue, param, Some(name), internal = true, countFailedValues = true) + } + + def createLongAccum(name: String): Accumulator[Long] = { + newMetric(0L, name, LongAccumulatorParam) + } + + def createIntAccum(name: String): Accumulator[Int] = { + newMetric(0, name, IntAccumulatorParam) + } + + def createBlocksAccum(name: String): Accumulator[Seq[(BlockId, BlockStatus)]] = { + newMetric(Nil, name, UpdatedBlockStatusesAccumulatorParam) } /** @@ -297,18 +279,11 @@ private[spark] object TaskMetrics extends Logging { * internal task level metrics. */ def fromAccumulatorUpdates(accumUpdates: Seq[AccumulableInfo]): TaskMetrics = { - // Initial accumulators are passed into the TaskMetrics constructor first because these - // are required to be uniquely named. The rest of the accumulators from this task are - // registered later because they need not satisfy this requirement. - val definedAccumUpdates = accumUpdates.filter { info => info.update.isDefined } - val initialAccums = definedAccumUpdates - .filter { info => info.name.exists(_.startsWith(InternalAccumulator.METRICS_PREFIX)) } - .map { info => - val accum = InternalAccumulator.create(info.name.get) - accum.setValueAny(info.update.get) - accum - } - new ListenerTaskMetrics(initialAccums, definedAccumUpdates) + val definedAccumUpdates = accumUpdates.filter(_.update.isDefined) + val metrics = new ListenerTaskMetrics(definedAccumUpdates) + definedAccumUpdates.filter(_.internal).foreach { accum => + metrics.internalAccums.find(_.name == accum.name).foreach(_.setValueAny(accum.update.get)) + } + metrics } - } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index c27aad268d32a..b7fb608ea5064 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1029,7 +1029,7 @@ class DAGScheduler( val locs = taskIdToLocations(id) val part = stage.rdd.partitions(id) new ShuffleMapTask(stage.id, stage.latestInfo.attemptId, - taskBinary, part, locs, stage.latestInfo.internalAccumulators, properties) + taskBinary, part, locs, stage.latestInfo.taskMetrics, properties) } case stage: ResultStage => @@ -1039,7 +1039,7 @@ class DAGScheduler( val part = stage.rdd.partitions(p) val locs = taskIdToLocations(id) new ResultTask(stage.id, stage.latestInfo.attemptId, - taskBinary, part, locs, id, properties, stage.latestInfo.internalAccumulators) + taskBinary, part, locs, id, properties, stage.latestInfo.taskMetrics) } } } catch { diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index db6276f75d781..75c6018e214d8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -23,6 +23,7 @@ import java.util.Properties import org.apache.spark._ import org.apache.spark.broadcast.Broadcast +import org.apache.spark.executor.TaskMetrics import org.apache.spark.rdd.RDD /** @@ -40,9 +41,7 @@ import org.apache.spark.rdd.RDD * @param outputId index of the task in this job (a job can launch tasks on only a subset of the * input RDD's partitions). * @param localProperties copy of thread-local properties set by the user on the driver side. - * @param _initialAccums initial set of accumulators to be used in this task for tracking - * internal metrics. Other accumulators will be registered later when - * they are deserialized on the executors. + * @param metrics a [[TaskMetrics]] that is created at driver side and sent to executor side. */ private[spark] class ResultTask[T, U]( stageId: Int, @@ -52,8 +51,8 @@ private[spark] class ResultTask[T, U]( locs: Seq[TaskLocation], val outputId: Int, localProperties: Properties, - _initialAccums: Seq[Accumulator[_]] = InternalAccumulator.createAll()) - extends Task[U](stageId, stageAttemptId, partition.index, _initialAccums, localProperties) + metrics: TaskMetrics) + extends Task[U](stageId, stageAttemptId, partition.index, metrics, localProperties) with Serializable { @transient private[this] val preferredLocs: Seq[TaskLocation] = { @@ -68,7 +67,6 @@ private[spark] class ResultTask[T, U]( ByteBuffer.wrap(taskBinary.value), Thread.currentThread.getContextClassLoader) _executorDeserializeTime = System.currentTimeMillis() - deserializeStartTime - metrics = Some(context.taskMetrics) func(context, rdd.iterator(partition, context)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index b7cab7013ef6f..84b3e5ba6c1f3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -24,6 +24,7 @@ import scala.language.existentials import org.apache.spark._ import org.apache.spark.broadcast.Broadcast +import org.apache.spark.executor.TaskMetrics import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.shuffle.ShuffleWriter @@ -40,9 +41,7 @@ import org.apache.spark.shuffle.ShuffleWriter * the type should be (RDD[_], ShuffleDependency[_, _, _]). * @param partition partition of the RDD this task is associated with * @param locs preferred task execution locations for locality scheduling - * @param _initialAccums initial set of accumulators to be used in this task for tracking - * internal metrics. Other accumulators will be registered later when - * they are deserialized on the executors. + * @param metrics a [[TaskMetrics]] that is created at driver side and sent to executor side. * @param localProperties copy of thread-local properties set by the user on the driver side. */ private[spark] class ShuffleMapTask( @@ -51,9 +50,9 @@ private[spark] class ShuffleMapTask( taskBinary: Broadcast[Array[Byte]], partition: Partition, @transient private var locs: Seq[TaskLocation], - _initialAccums: Seq[Accumulator[_]], + metrics: TaskMetrics, localProperties: Properties) - extends Task[MapStatus](stageId, stageAttemptId, partition.index, _initialAccums, localProperties) + extends Task[MapStatus](stageId, stageAttemptId, partition.index, metrics, localProperties) with Logging { /** A constructor used only in test suites. This does not require passing in an RDD. */ @@ -73,7 +72,6 @@ private[spark] class ShuffleMapTask( ByteBuffer.wrap(taskBinary.value), Thread.currentThread.getContextClassLoader) _executorDeserializeTime = System.currentTimeMillis() - deserializeStartTime - metrics = Some(context.taskMetrics) var writer: ShuffleWriter[Any, Any] = null try { val manager = SparkEnv.get.shuffleManager diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index b6d4e39fe532a..d5cf6b82e86f0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler import scala.collection.mutable.HashSet import org.apache.spark._ +import org.apache.spark.executor.TaskMetrics import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.util.CallSite @@ -110,9 +111,10 @@ private[scheduler] abstract class Stage( def makeNewStageAttempt( numPartitionsToCompute: Int, taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty): Unit = { + val metrics = new TaskMetrics + metrics.registerAccums(rdd.sparkContext) _latestInfo = StageInfo.fromStage( - this, nextAttemptId, Some(numPartitionsToCompute), - InternalAccumulator.createAll(rdd.sparkContext), taskLocalityPreferences) + this, nextAttemptId, Some(numPartitionsToCompute), metrics, taskLocalityPreferences) nextAttemptId += 1 } diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 0fd58c41cdceb..58349fe250887 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -19,8 +19,8 @@ package org.apache.spark.scheduler import scala.collection.mutable.HashMap -import org.apache.spark.Accumulator import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.RDDInfo /** @@ -36,7 +36,7 @@ class StageInfo( val rddInfos: Seq[RDDInfo], val parentIds: Seq[Int], val details: String, - val internalAccumulators: Seq[Accumulator[_]] = Seq.empty, + val taskMetrics: TaskMetrics = new TaskMetrics, private[spark] val taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None @@ -81,7 +81,7 @@ private[spark] object StageInfo { stage: Stage, attemptId: Int, numTasks: Option[Int] = None, - internalAccumulators: Seq[Accumulator[_]] = Seq.empty, + taskMetrics: TaskMetrics = new TaskMetrics, taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty ): StageInfo = { val ancestorRddInfos = stage.rdd.getNarrowAncestors.map(RDDInfo.fromRdd) @@ -94,7 +94,7 @@ private[spark] object StageInfo { rddInfos, stage.parents.map(_.id), stage.details, - internalAccumulators, + taskMetrics, taskLocalityPreferences) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 1ff9d7795f42e..9f2fa02c69ab1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -23,7 +23,7 @@ import java.util.Properties import scala.collection.mutable.HashMap -import org.apache.spark.{Accumulator, SparkEnv, TaskContext, TaskContextImpl} +import org.apache.spark.{SparkEnv, TaskContext, TaskContextImpl} import org.apache.spark.executor.TaskMetrics import org.apache.spark.memory.{MemoryMode, TaskMemoryManager} import org.apache.spark.metrics.MetricsSystem @@ -44,17 +44,17 @@ import org.apache.spark.util.{ByteBufferInputStream, ByteBufferOutputStream, Uti * @param stageId id of the stage this task belongs to * @param stageAttemptId attempt id of the stage this task belongs to * @param partitionId index of the number in the RDD - * @param initialAccumulators initial set of accumulators to be used in this task for tracking - * internal metrics. Other accumulators will be registered later when - * they are deserialized on the executors. + * @param metrics a [[TaskMetrics]] that is created at driver side and sent to executor side. * @param localProperties copy of thread-local properties set by the user on the driver side. + * + * The default values for `metrics` and `localProperties` are used by tests only. */ private[spark] abstract class Task[T]( val stageId: Int, val stageAttemptId: Int, val partitionId: Int, - val initialAccumulators: Seq[Accumulator[_]], - @transient var localProperties: Properties) extends Serializable { + val metrics: TaskMetrics = new TaskMetrics, + @transient var localProperties: Properties = new Properties) extends Serializable { /** * Called by [[org.apache.spark.executor.Executor]] to run this task. @@ -76,7 +76,7 @@ private[spark] abstract class Task[T]( taskMemoryManager, localProperties, metricsSystem, - initialAccumulators) + metrics) TaskContext.setTaskContext(context) taskThread = Thread.currentThread() if (_killed) { @@ -128,8 +128,6 @@ private[spark] abstract class Task[T]( // Map output tracker epoch. Will be set by TaskScheduler. var epoch: Long = -1 - var metrics: Option[TaskMetrics] = None - // Task context, to be initialized in run(). @transient protected var context: TaskContextImpl = _ diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index bd4797ae8e0c5..645e2d2e360bb 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -203,7 +203,7 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { // This could be empty if the JobProgressListener hasn't received information about the // stage or if the stage information has been garbage collected listener.stageIdToInfo.getOrElse(stageId, - new StageInfo(stageId, 0, "Unknown", 0, Seq.empty, Seq.empty, "Unknown", Seq.empty)) + new StageInfo(stageId, 0, "Unknown", 0, Seq.empty, Seq.empty, "Unknown")) } val activeStages = Buffer[StageInfo]() diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 38ca3224ffe19..6c50c72a91ef2 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -304,20 +304,17 @@ private[spark] object JsonProtocol { * The behavior here must match that of [[accumValueFromJson]]. Exposed for testing. */ private[util] def accumValueToJson(name: Option[String], value: Any): JValue = { - import AccumulatorParam._ if (name.exists(_.startsWith(InternalAccumulator.METRICS_PREFIX))) { - (value, InternalAccumulator.getParam(name.get)) match { - case (v: Int, IntAccumulatorParam) => JInt(v) - case (v: Long, LongAccumulatorParam) => JInt(v) - case (v: String, StringAccumulatorParam) => JString(v) - case (v, UpdatedBlockStatusesAccumulatorParam) => + value match { + case v: Int => JInt(v) + case v: Long => JInt(v) + // We only have 3 kind of internal accumulator types, so if it's not int or long, it must be + // the blocks accumulator, whose type is `Seq[(BlockId, BlockStatus)]` + case v => JArray(v.asInstanceOf[Seq[(BlockId, BlockStatus)]].toList.map { case (id, status) => ("Block ID" -> id.toString) ~ ("Status" -> blockStatusToJson(status)) }) - case (v, p) => - throw new IllegalArgumentException(s"unexpected combination of accumulator value " + - s"type (${v.getClass.getName}) and param (${p.getClass.getName}) in '${name.get}'") } } else { // For all external accumulators, just use strings @@ -569,7 +566,7 @@ private[spark] object JsonProtocol { val stageInfos = Utils.jsonOption(json \ "Stage Infos") .map(_.extract[Seq[JValue]].map(stageInfoFromJson)).getOrElse { stageIds.map { id => - new StageInfo(id, 0, "unknown", 0, Seq.empty, Seq.empty, "unknown", Seq.empty) + new StageInfo(id, 0, "unknown", 0, Seq.empty, Seq.empty, "unknown") } } SparkListenerJobStart(jobId, submissionTime, stageInfos, properties) @@ -678,7 +675,7 @@ private[spark] object JsonProtocol { } val stageInfo = new StageInfo( - stageId, attemptId, stageName, numTasks, rddInfos, parentIds, details, Seq.empty) + stageId, attemptId, stageName, numTasks, rddInfos, parentIds, details) stageInfo.submissionTime = submissionTime stageInfo.completionTime = completionTime stageInfo.failureReason = failureReason @@ -735,25 +732,21 @@ private[spark] object JsonProtocol { * The behavior here must match that of [[accumValueToJson]]. Exposed for testing. */ private[util] def accumValueFromJson(name: Option[String], value: JValue): Any = { - import AccumulatorParam._ if (name.exists(_.startsWith(InternalAccumulator.METRICS_PREFIX))) { - (value, InternalAccumulator.getParam(name.get)) match { - case (JInt(v), IntAccumulatorParam) => v.toInt - case (JInt(v), LongAccumulatorParam) => v.toLong - case (JString(v), StringAccumulatorParam) => v - case (JArray(v), UpdatedBlockStatusesAccumulatorParam) => + value match { + case JInt(v) => v.toLong + case JArray(v) => v.map { blockJson => val id = BlockId((blockJson \ "Block ID").extract[String]) val status = blockStatusFromJson(blockJson \ "Status") (id, status) } - case (v, p) => - throw new IllegalArgumentException(s"unexpected combination of accumulator " + - s"value in JSON ($v) and accumulator param (${p.getClass.getName}) in '${name.get}'") - } - } else { - value.extract[String] - } + case _ => throw new IllegalArgumentException(s"unexpected json value $value for " + + "accumulator " + name.get) + } + } else { + value.extract[String] + } } def taskMetricsFromJson(json: JValue): TaskMetrics = { diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 37879d11caec4..454c42517ca1b 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark -import java.util.Properties import java.util.concurrent.Semaphore import javax.annotation.concurrent.GuardedBy @@ -29,6 +28,7 @@ import scala.util.control.NonFatal import org.scalatest.Matchers import org.scalatest.exceptions.TestFailedException +import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ import org.apache.spark.serializer.JavaSerializer @@ -278,16 +278,13 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex val acc1 = new Accumulator(0, IntAccumulatorParam, Some("thing"), internal = false) val acc2 = new Accumulator(0L, LongAccumulatorParam, Some("thing2"), internal = false) val externalAccums = Seq(acc1, acc2) - val internalAccums = InternalAccumulator.createAll() + val taskMetrics = new TaskMetrics // Set some values; these should not be observed later on the "executors" acc1.setValue(10) acc2.setValue(20L) - internalAccums - .find(_.name == Some(InternalAccumulator.TEST_ACCUM)) - .get.asInstanceOf[Accumulator[Long]] - .setValue(30L) + taskMetrics.testAccum.get.asInstanceOf[Accumulator[Long]].setValue(30L) // Simulate the task being serialized and sent to the executors. - val dummyTask = new DummyTask(internalAccums, externalAccums) + val dummyTask = new DummyTask(taskMetrics, externalAccums) val serInstance = new JavaSerializer(new SparkConf).newInstance() val taskSer = Task.serializeWithDependencies( dummyTask, mutable.HashMap(), mutable.HashMap(), serInstance) @@ -298,7 +295,7 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex taskBytes, Thread.currentThread.getContextClassLoader) // Assert that executors see only zeros taskDeser.externalAccums.foreach { a => assert(a.localValue == a.zero) } - taskDeser.internalAccums.foreach { a => assert(a.localValue == a.zero) } + taskDeser.metrics.internalAccums.foreach { a => assert(a.localValue == a.zero) } } } @@ -402,8 +399,7 @@ private class SaveInfoListener extends SparkListener { * A dummy [[Task]] that contains internal and external [[Accumulator]]s. */ private[spark] class DummyTask( - val internalAccums: Seq[Accumulator[_]], - val externalAccums: Seq[Accumulator[_]]) - extends Task[Int](0, 0, 0, internalAccums, new Properties) { + metrics: TaskMetrics, + val externalAccums: Seq[Accumulator[_]]) extends Task[Int](0, 0, 0, metrics) { override def runTask(c: TaskContext): Int = 1 } diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index ee6b991461902..c130649830416 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -929,7 +929,7 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty ): StageInfo = { new StageInfo(stageId, 0, "name", numTasks, Seq.empty, Seq.empty, "no details", - Seq.empty, taskLocalityPreferences) + taskLocalityPreferences = taskLocalityPreferences) } private def createTaskInfo(taskId: Int, taskIndex: Int, executorId: String): TaskInfo = { diff --git a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala index db087a9c3c0d1..b074b95424731 100644 --- a/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/InternalAccumulatorSuite.scala @@ -19,14 +19,13 @@ package org.apache.spark import scala.collection.mutable.ArrayBuffer +import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.shuffle.FetchFailedException -import org.apache.spark.storage.{BlockId, BlockStatus} class InternalAccumulatorSuite extends SparkFunSuite with LocalSparkContext { import InternalAccumulator._ - import AccumulatorParam._ override def afterEach(): Unit = { try { @@ -36,120 +35,12 @@ class InternalAccumulatorSuite extends SparkFunSuite with LocalSparkContext { } } - test("get param") { - assert(getParam(EXECUTOR_DESERIALIZE_TIME) === LongAccumulatorParam) - assert(getParam(EXECUTOR_RUN_TIME) === LongAccumulatorParam) - assert(getParam(RESULT_SIZE) === LongAccumulatorParam) - assert(getParam(JVM_GC_TIME) === LongAccumulatorParam) - assert(getParam(RESULT_SERIALIZATION_TIME) === LongAccumulatorParam) - assert(getParam(MEMORY_BYTES_SPILLED) === LongAccumulatorParam) - assert(getParam(DISK_BYTES_SPILLED) === LongAccumulatorParam) - assert(getParam(PEAK_EXECUTION_MEMORY) === LongAccumulatorParam) - assert(getParam(UPDATED_BLOCK_STATUSES) === UpdatedBlockStatusesAccumulatorParam) - assert(getParam(TEST_ACCUM) === LongAccumulatorParam) - // shuffle read - assert(getParam(shuffleRead.REMOTE_BLOCKS_FETCHED) === IntAccumulatorParam) - assert(getParam(shuffleRead.LOCAL_BLOCKS_FETCHED) === IntAccumulatorParam) - assert(getParam(shuffleRead.REMOTE_BYTES_READ) === LongAccumulatorParam) - assert(getParam(shuffleRead.LOCAL_BYTES_READ) === LongAccumulatorParam) - assert(getParam(shuffleRead.FETCH_WAIT_TIME) === LongAccumulatorParam) - assert(getParam(shuffleRead.RECORDS_READ) === LongAccumulatorParam) - // shuffle write - assert(getParam(shuffleWrite.BYTES_WRITTEN) === LongAccumulatorParam) - assert(getParam(shuffleWrite.RECORDS_WRITTEN) === LongAccumulatorParam) - assert(getParam(shuffleWrite.WRITE_TIME) === LongAccumulatorParam) - // input - assert(getParam(input.RECORDS_READ) === LongAccumulatorParam) - assert(getParam(input.BYTES_READ) === LongAccumulatorParam) - // output - assert(getParam(output.RECORDS_WRITTEN) === LongAccumulatorParam) - assert(getParam(output.BYTES_WRITTEN) === LongAccumulatorParam) - // default to Long - assert(getParam(METRICS_PREFIX + "anything") === LongAccumulatorParam) - intercept[IllegalArgumentException] { - getParam("something that does not start with the right prefix") - } - } - - test("create by name") { - val executorRunTime = create(EXECUTOR_RUN_TIME) - val updatedBlockStatuses = create(UPDATED_BLOCK_STATUSES) - val shuffleRemoteBlocksRead = create(shuffleRead.REMOTE_BLOCKS_FETCHED) - assert(executorRunTime.name === Some(EXECUTOR_RUN_TIME)) - assert(updatedBlockStatuses.name === Some(UPDATED_BLOCK_STATUSES)) - assert(shuffleRemoteBlocksRead.name === Some(shuffleRead.REMOTE_BLOCKS_FETCHED)) - assert(executorRunTime.value.isInstanceOf[Long]) - assert(updatedBlockStatuses.value.isInstanceOf[Seq[_]]) - // We cannot assert the type of the value directly since the type parameter is erased. - // Instead, try casting a `Seq` of expected type and see if it fails in run time. - updatedBlockStatuses.setValueAny(Seq.empty[(BlockId, BlockStatus)]) - assert(shuffleRemoteBlocksRead.value.isInstanceOf[Int]) - // default to Long - val anything = create(METRICS_PREFIX + "anything") - assert(anything.value.isInstanceOf[Long]) - } - - test("create") { - val accums = createAll() - val shuffleReadAccums = createShuffleReadAccums() - val shuffleWriteAccums = createShuffleWriteAccums() - val inputAccums = createInputAccums() - val outputAccums = createOutputAccums() - // assert they're all internal - assert(accums.forall(_.isInternal)) - assert(shuffleReadAccums.forall(_.isInternal)) - assert(shuffleWriteAccums.forall(_.isInternal)) - assert(inputAccums.forall(_.isInternal)) - assert(outputAccums.forall(_.isInternal)) - // assert they all count on failures - assert(accums.forall(_.countFailedValues)) - assert(shuffleReadAccums.forall(_.countFailedValues)) - assert(shuffleWriteAccums.forall(_.countFailedValues)) - assert(inputAccums.forall(_.countFailedValues)) - assert(outputAccums.forall(_.countFailedValues)) - // assert they all have names - assert(accums.forall(_.name.isDefined)) - assert(shuffleReadAccums.forall(_.name.isDefined)) - assert(shuffleWriteAccums.forall(_.name.isDefined)) - assert(inputAccums.forall(_.name.isDefined)) - assert(outputAccums.forall(_.name.isDefined)) - // assert `accums` is a strict superset of the others - val accumNames = accums.map(_.name.get).toSet - val shuffleReadAccumNames = shuffleReadAccums.map(_.name.get).toSet - val shuffleWriteAccumNames = shuffleWriteAccums.map(_.name.get).toSet - val inputAccumNames = inputAccums.map(_.name.get).toSet - val outputAccumNames = outputAccums.map(_.name.get).toSet - assert(shuffleReadAccumNames.subsetOf(accumNames)) - assert(shuffleWriteAccumNames.subsetOf(accumNames)) - assert(inputAccumNames.subsetOf(accumNames)) - assert(outputAccumNames.subsetOf(accumNames)) - } - - test("naming") { - val accums = createAll() - val shuffleReadAccums = createShuffleReadAccums() - val shuffleWriteAccums = createShuffleWriteAccums() - val inputAccums = createInputAccums() - val outputAccums = createOutputAccums() - // assert that prefixes are properly namespaced - assert(SHUFFLE_READ_METRICS_PREFIX.startsWith(METRICS_PREFIX)) - assert(SHUFFLE_WRITE_METRICS_PREFIX.startsWith(METRICS_PREFIX)) - assert(INPUT_METRICS_PREFIX.startsWith(METRICS_PREFIX)) - assert(OUTPUT_METRICS_PREFIX.startsWith(METRICS_PREFIX)) - assert(accums.forall(_.name.get.startsWith(METRICS_PREFIX))) - // assert they all start with the expected prefixes - assert(shuffleReadAccums.forall(_.name.get.startsWith(SHUFFLE_READ_METRICS_PREFIX))) - assert(shuffleWriteAccums.forall(_.name.get.startsWith(SHUFFLE_WRITE_METRICS_PREFIX))) - assert(inputAccums.forall(_.name.get.startsWith(INPUT_METRICS_PREFIX))) - assert(outputAccums.forall(_.name.get.startsWith(OUTPUT_METRICS_PREFIX))) - } - test("internal accumulators in TaskContext") { val taskContext = TaskContext.empty() val accumUpdates = taskContext.taskMetrics.accumulatorUpdates() assert(accumUpdates.size > 0) assert(accumUpdates.forall(_.internal)) - val testAccum = taskContext.taskMetrics.getAccum(TEST_ACCUM) + val testAccum = taskContext.taskMetrics.testAccum.get assert(accumUpdates.exists(_.id == testAccum.id)) } @@ -160,7 +51,7 @@ class InternalAccumulatorSuite extends SparkFunSuite with LocalSparkContext { sc.addSparkListener(listener) // Have each task add 1 to the internal accumulator val rdd = sc.parallelize(1 to 100, numPartitions).mapPartitions { iter => - TaskContext.get().taskMetrics().getAccum(TEST_ACCUM) += 1 + TaskContext.get().taskMetrics().testAccum.get += 1 iter } // Register asserts in job completion callback to avoid flakiness @@ -196,17 +87,17 @@ class InternalAccumulatorSuite extends SparkFunSuite with LocalSparkContext { val rdd = sc.parallelize(1 to 100, numPartitions) .map { i => (i, i) } .mapPartitions { iter => - TaskContext.get().taskMetrics().getAccum(TEST_ACCUM) += 1 + TaskContext.get().taskMetrics().testAccum.get += 1 iter } .reduceByKey { case (x, y) => x + y } .mapPartitions { iter => - TaskContext.get().taskMetrics().getAccum(TEST_ACCUM) += 10 + TaskContext.get().taskMetrics().testAccum.get += 10 iter } .repartition(numPartitions * 2) .mapPartitions { iter => - TaskContext.get().taskMetrics().getAccum(TEST_ACCUM) += 100 + TaskContext.get().taskMetrics().testAccum.get += 100 iter } // Register asserts in job completion callback to avoid flakiness @@ -236,7 +127,7 @@ class InternalAccumulatorSuite extends SparkFunSuite with LocalSparkContext { // This should retry both stages in the scheduler. Note that we only want to fail the // first stage attempt because we want the stage to eventually succeed. val x = sc.parallelize(1 to 100, numPartitions) - .mapPartitions { iter => TaskContext.get().taskMetrics().getAccum(TEST_ACCUM) += 1; iter } + .mapPartitions { iter => TaskContext.get().taskMetrics().testAccum.get += 1; iter } .groupBy(identity) val sid = x.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleHandle.shuffleId val rdd = x.mapPartitionsWithIndex { case (i, iter) => @@ -294,15 +185,15 @@ class InternalAccumulatorSuite extends SparkFunSuite with LocalSparkContext { } assert(Accumulators.originals.isEmpty) sc.parallelize(1 to 100).map { i => (i, i) }.reduceByKey { _ + _ }.count() - val internalAccums = InternalAccumulator.createAll() + val numInternalAccums = TaskMetrics.empty.internalAccums.length // We ran 2 stages, so we should have 2 sets of internal accumulators, 1 for each stage - assert(Accumulators.originals.size === internalAccums.size * 2) + assert(Accumulators.originals.size === numInternalAccums * 2) val accumsRegistered = sc.cleaner match { case Some(cleaner: SaveAccumContextCleaner) => cleaner.accumsRegisteredForCleanup case _ => Seq.empty[Long] } // Make sure the same set of accumulators is registered for cleanup - assert(accumsRegistered.size === internalAccums.size * 2) + assert(accumsRegistered.size === numInternalAccums * 2) assert(accumsRegistered.toSet === Accumulators.originals.keys.toSet) } diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 079109d137065..a854f5bb9b7ce 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -336,16 +336,14 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC // first attempt -- its successful val writer1 = manager.getWriter[Int, Int](shuffleHandle, 0, - new TaskContextImpl(0, 0, 0L, 0, taskMemoryManager, new Properties, metricsSystem, - InternalAccumulator.createAll(sc))) + new TaskContextImpl(0, 0, 0L, 0, taskMemoryManager, new Properties, metricsSystem)) val data1 = (1 to 10).map { x => x -> x} // second attempt -- also successful. We'll write out different data, // just to simulate the fact that the records may get written differently // depending on what gets spilled, what gets combined, etc. val writer2 = manager.getWriter[Int, Int](shuffleHandle, 0, - new TaskContextImpl(0, 0, 1L, 0, taskMemoryManager, new Properties, metricsSystem, - InternalAccumulator.createAll(sc))) + new TaskContextImpl(0, 0, 1L, 0, taskMemoryManager, new Properties, metricsSystem)) val data2 = (11 to 20).map { x => x -> x} // interleave writes of both attempts -- we want to test that both attempts can occur @@ -373,8 +371,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC } val reader = manager.getReader[Int, Int](shuffleHandle, 0, 1, - new TaskContextImpl(1, 0, 2L, 0, taskMemoryManager, new Properties, metricsSystem, - InternalAccumulator.createAll(sc))) + new TaskContextImpl(1, 0, 2L, 0, taskMemoryManager, new Properties, metricsSystem)) val readData = reader.read().toIndexedSeq assert(readData === data1.toIndexedSeq || readData === data2.toIndexedSeq) diff --git a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala index a263fce8abd9a..fbc2fae08df24 100644 --- a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala @@ -26,102 +26,20 @@ import org.apache.spark.storage.{BlockId, BlockStatus, StorageLevel, TestBlockId class TaskMetricsSuite extends SparkFunSuite { import AccumulatorParam._ - import InternalAccumulator._ import StorageLevel._ import TaskMetricsSuite._ - test("create with unnamed accum") { - intercept[IllegalArgumentException] { - new TaskMetrics( - InternalAccumulator.createAll() ++ Seq( - new Accumulator(0, IntAccumulatorParam, None, internal = true))) - } - } - - test("create with duplicate name accum") { - intercept[IllegalArgumentException] { - new TaskMetrics( - InternalAccumulator.createAll() ++ Seq( - new Accumulator(0, IntAccumulatorParam, Some(RESULT_SIZE), internal = true))) - } - } - - test("create with external accum") { - intercept[IllegalArgumentException] { - new TaskMetrics( - InternalAccumulator.createAll() ++ Seq( - new Accumulator(0, IntAccumulatorParam, Some("x")))) - } - } - - test("create shuffle read metrics") { - import shuffleRead._ - val accums = InternalAccumulator.createShuffleReadAccums() - .map { a => (a.name.get, a) }.toMap[String, Accumulator[_]] - accums(REMOTE_BLOCKS_FETCHED).setValueAny(1) - accums(LOCAL_BLOCKS_FETCHED).setValueAny(2) - accums(REMOTE_BYTES_READ).setValueAny(3L) - accums(LOCAL_BYTES_READ).setValueAny(4L) - accums(FETCH_WAIT_TIME).setValueAny(5L) - accums(RECORDS_READ).setValueAny(6L) - val sr = new ShuffleReadMetrics(accums) - assert(sr.remoteBlocksFetched === 1) - assert(sr.localBlocksFetched === 2) - assert(sr.remoteBytesRead === 3L) - assert(sr.localBytesRead === 4L) - assert(sr.fetchWaitTime === 5L) - assert(sr.recordsRead === 6L) - } - - test("create shuffle write metrics") { - import shuffleWrite._ - val accums = InternalAccumulator.createShuffleWriteAccums() - .map { a => (a.name.get, a) }.toMap[String, Accumulator[_]] - accums(BYTES_WRITTEN).setValueAny(1L) - accums(RECORDS_WRITTEN).setValueAny(2L) - accums(WRITE_TIME).setValueAny(3L) - val sw = new ShuffleWriteMetrics(accums) - assert(sw.bytesWritten === 1L) - assert(sw.recordsWritten === 2L) - assert(sw.writeTime === 3L) - } - - test("create input metrics") { - import input._ - val accums = InternalAccumulator.createInputAccums() - .map { a => (a.name.get, a) }.toMap[String, Accumulator[_]] - accums(BYTES_READ).setValueAny(1L) - accums(RECORDS_READ).setValueAny(2L) - val im = new InputMetrics(accums) - assert(im.bytesRead === 1L) - assert(im.recordsRead === 2L) - } - - test("create output metrics") { - import output._ - val accums = InternalAccumulator.createOutputAccums() - .map { a => (a.name.get, a) }.toMap[String, Accumulator[_]] - accums(BYTES_WRITTEN).setValueAny(1L) - accums(RECORDS_WRITTEN).setValueAny(2L) - val om = new OutputMetrics(accums) - assert(om.bytesWritten === 1L) - assert(om.recordsWritten === 2L) - } - test("mutating values") { - val accums = InternalAccumulator.createAll() - val tm = new TaskMetrics(accums) - // initial values - assertValueEquals(tm, _.executorDeserializeTime, accums, EXECUTOR_DESERIALIZE_TIME, 0L) - assertValueEquals(tm, _.executorRunTime, accums, EXECUTOR_RUN_TIME, 0L) - assertValueEquals(tm, _.resultSize, accums, RESULT_SIZE, 0L) - assertValueEquals(tm, _.jvmGCTime, accums, JVM_GC_TIME, 0L) - assertValueEquals(tm, _.resultSerializationTime, accums, RESULT_SERIALIZATION_TIME, 0L) - assertValueEquals(tm, _.memoryBytesSpilled, accums, MEMORY_BYTES_SPILLED, 0L) - assertValueEquals(tm, _.diskBytesSpilled, accums, DISK_BYTES_SPILLED, 0L) - assertValueEquals(tm, _.peakExecutionMemory, accums, PEAK_EXECUTION_MEMORY, 0L) - assertValueEquals(tm, _.updatedBlockStatuses, accums, UPDATED_BLOCK_STATUSES, - Seq.empty[(BlockId, BlockStatus)]) + val tm = new TaskMetrics + assert(tm.executorDeserializeTime == 0L) + assert(tm.executorRunTime == 0L) + assert(tm.resultSize == 0L) + assert(tm.jvmGCTime == 0L) + assert(tm.resultSerializationTime == 0L) + assert(tm.memoryBytesSpilled == 0L) + assert(tm.diskBytesSpilled == 0L) + assert(tm.peakExecutionMemory == 0L) + assert(tm.updatedBlockStatuses.isEmpty) // set or increment values tm.setExecutorDeserializeTime(100L) tm.setExecutorDeserializeTime(1L) // overwrite @@ -144,36 +62,27 @@ class TaskMetricsSuite extends SparkFunSuite { tm.incUpdatedBlockStatuses(Seq(block1)) tm.incUpdatedBlockStatuses(Seq(block2)) // assert new values exist - assertValueEquals(tm, _.executorDeserializeTime, accums, EXECUTOR_DESERIALIZE_TIME, 1L) - assertValueEquals(tm, _.executorRunTime, accums, EXECUTOR_RUN_TIME, 2L) - assertValueEquals(tm, _.resultSize, accums, RESULT_SIZE, 3L) - assertValueEquals(tm, _.jvmGCTime, accums, JVM_GC_TIME, 4L) - assertValueEquals(tm, _.resultSerializationTime, accums, RESULT_SERIALIZATION_TIME, 5L) - assertValueEquals(tm, _.memoryBytesSpilled, accums, MEMORY_BYTES_SPILLED, 606L) - assertValueEquals(tm, _.diskBytesSpilled, accums, DISK_BYTES_SPILLED, 707L) - assertValueEquals(tm, _.peakExecutionMemory, accums, PEAK_EXECUTION_MEMORY, 808L) - assertValueEquals(tm, _.updatedBlockStatuses, accums, UPDATED_BLOCK_STATUSES, - Seq(block1, block2)) + assert(tm.executorDeserializeTime == 1L) + assert(tm.executorRunTime == 2L) + assert(tm.resultSize == 3L) + assert(tm.jvmGCTime == 4L) + assert(tm.resultSerializationTime == 5L) + assert(tm.memoryBytesSpilled == 606L) + assert(tm.diskBytesSpilled == 707L) + assert(tm.peakExecutionMemory == 808L) + assert(tm.updatedBlockStatuses == Seq(block1, block2)) } test("mutating shuffle read metrics values") { - import shuffleRead._ - val accums = InternalAccumulator.createAll() - val tm = new TaskMetrics(accums) - def assertValEquals[T](tmValue: ShuffleReadMetrics => T, name: String, value: T): Unit = { - assertValueEquals(tm, tm => tmValue(tm.shuffleReadMetrics), accums, name, value) - } - // create shuffle read metrics - tm.createTempShuffleReadMetrics() - tm.mergeShuffleReadMetrics() + val tm = new TaskMetrics val sr = tm.shuffleReadMetrics // initial values - assertValEquals(_.remoteBlocksFetched, REMOTE_BLOCKS_FETCHED, 0) - assertValEquals(_.localBlocksFetched, LOCAL_BLOCKS_FETCHED, 0) - assertValEquals(_.remoteBytesRead, REMOTE_BYTES_READ, 0L) - assertValEquals(_.localBytesRead, LOCAL_BYTES_READ, 0L) - assertValEquals(_.fetchWaitTime, FETCH_WAIT_TIME, 0L) - assertValEquals(_.recordsRead, RECORDS_READ, 0L) + assert(sr.remoteBlocksFetched == 0) + assert(sr.localBlocksFetched == 0) + assert(sr.remoteBytesRead == 0L) + assert(sr.localBytesRead == 0L) + assert(sr.fetchWaitTime == 0L) + assert(sr.recordsRead == 0L) // set and increment values sr.setRemoteBlocksFetched(100) sr.setRemoteBlocksFetched(10) @@ -200,27 +109,21 @@ class TaskMetricsSuite extends SparkFunSuite { sr.incRecordsRead(6L) sr.incRecordsRead(6L) // assert new values exist - assertValEquals(_.remoteBlocksFetched, REMOTE_BLOCKS_FETCHED, 12) - assertValEquals(_.localBlocksFetched, LOCAL_BLOCKS_FETCHED, 24) - assertValEquals(_.remoteBytesRead, REMOTE_BYTES_READ, 36L) - assertValEquals(_.localBytesRead, LOCAL_BYTES_READ, 48L) - assertValEquals(_.fetchWaitTime, FETCH_WAIT_TIME, 60L) - assertValEquals(_.recordsRead, RECORDS_READ, 72L) + assert(sr.remoteBlocksFetched == 12) + assert(sr.localBlocksFetched == 24) + assert(sr.remoteBytesRead == 36L) + assert(sr.localBytesRead == 48L) + assert(sr.fetchWaitTime == 60L) + assert(sr.recordsRead == 72L) } test("mutating shuffle write metrics values") { - import shuffleWrite._ - val accums = InternalAccumulator.createAll() - val tm = new TaskMetrics(accums) - def assertValEquals[T](tmValue: ShuffleWriteMetrics => T, name: String, value: T): Unit = { - assertValueEquals(tm, tm => tmValue(tm.shuffleWriteMetrics), accums, name, value) - } - // create shuffle write metrics + val tm = new TaskMetrics val sw = tm.shuffleWriteMetrics // initial values - assertValEquals(_.bytesWritten, BYTES_WRITTEN, 0L) - assertValEquals(_.recordsWritten, RECORDS_WRITTEN, 0L) - assertValEquals(_.writeTime, WRITE_TIME, 0L) + assert(sw.bytesWritten == 0L) + assert(sw.recordsWritten == 0L) + assert(sw.writeTime == 0L) // increment and decrement values sw.incBytesWritten(100L) sw.incBytesWritten(10L) // 100 + 10 @@ -233,55 +136,41 @@ class TaskMetricsSuite extends SparkFunSuite { sw.incWriteTime(300L) sw.incWriteTime(30L) // assert new values exist - assertValEquals(_.bytesWritten, BYTES_WRITTEN, 108L) - assertValEquals(_.recordsWritten, RECORDS_WRITTEN, 216L) - assertValEquals(_.writeTime, WRITE_TIME, 330L) + assert(sw.bytesWritten == 108L) + assert(sw.recordsWritten == 216L) + assert(sw.writeTime == 330L) } test("mutating input metrics values") { - import input._ - val accums = InternalAccumulator.createAll() - val tm = new TaskMetrics(accums) - def assertValEquals(tmValue: InputMetrics => Any, name: String, value: Any): Unit = { - assertValueEquals(tm, tm => tmValue(tm.inputMetrics), accums, name, value, - (x: Any, y: Any) => assert(x.toString === y.toString)) - } - // create input metrics + val tm = new TaskMetrics val in = tm.inputMetrics // initial values - assertValEquals(_.bytesRead, BYTES_READ, 0L) - assertValEquals(_.recordsRead, RECORDS_READ, 0L) + assert(in.bytesRead == 0L) + assert(in.recordsRead == 0L) // set and increment values in.setBytesRead(1L) in.setBytesRead(2L) in.incRecordsRead(1L) in.incRecordsRead(2L) // assert new values exist - assertValEquals(_.bytesRead, BYTES_READ, 2L) - assertValEquals(_.recordsRead, RECORDS_READ, 3L) + assert(in.bytesRead == 2L) + assert(in.recordsRead == 3L) } test("mutating output metrics values") { - import output._ - val accums = InternalAccumulator.createAll() - val tm = new TaskMetrics(accums) - def assertValEquals(tmValue: OutputMetrics => Any, name: String, value: Any): Unit = { - assertValueEquals(tm, tm => tmValue(tm.outputMetrics), accums, name, value, - (x: Any, y: Any) => assert(x.toString === y.toString)) - } - // create input metrics + val tm = new TaskMetrics val out = tm.outputMetrics // initial values - assertValEquals(_.bytesWritten, BYTES_WRITTEN, 0L) - assertValEquals(_.recordsWritten, RECORDS_WRITTEN, 0L) + assert(out.bytesWritten == 0L) + assert(out.recordsWritten == 0L) // set values out.setBytesWritten(1L) out.setBytesWritten(2L) out.setRecordsWritten(3L) out.setRecordsWritten(4L) // assert new values exist - assertValEquals(_.bytesWritten, BYTES_WRITTEN, 2L) - assertValEquals(_.recordsWritten, RECORDS_WRITTEN, 4L) + assert(out.bytesWritten == 2L) + assert(out.recordsWritten == 4L) } test("merging multiple shuffle read metrics") { @@ -305,9 +194,7 @@ class TaskMetricsSuite extends SparkFunSuite { } test("additional accumulables") { - val internalAccums = InternalAccumulator.createAll() - val tm = new TaskMetrics(internalAccums) - assert(tm.accumulatorUpdates().size === internalAccums.size) + val tm = new TaskMetrics val acc1 = new Accumulator(0, IntAccumulatorParam, Some("a")) val acc2 = new Accumulator(0, IntAccumulatorParam, Some("b")) val acc3 = new Accumulator(0, IntAccumulatorParam, Some("c")) @@ -338,47 +225,11 @@ class TaskMetricsSuite extends SparkFunSuite { assert(newUpdates(acc4.id).countFailedValues) assert(newUpdates.values.map(_.update).forall(_.isDefined)) assert(newUpdates.values.map(_.value).forall(_.isEmpty)) - assert(newUpdates.size === internalAccums.size + 4) - } - - test("existing values in shuffle read accums") { - // set shuffle read accum before passing it into TaskMetrics - val accums = InternalAccumulator.createAll() - val srAccum = accums.find(_.name === Some(shuffleRead.FETCH_WAIT_TIME)) - assert(srAccum.isDefined) - srAccum.get.asInstanceOf[Accumulator[Long]] += 10L - val tm = new TaskMetrics(accums) - } - - test("existing values in shuffle write accums") { - // set shuffle write accum before passing it into TaskMetrics - val accums = InternalAccumulator.createAll() - val swAccum = accums.find(_.name === Some(shuffleWrite.RECORDS_WRITTEN)) - assert(swAccum.isDefined) - swAccum.get.asInstanceOf[Accumulator[Long]] += 10L - val tm = new TaskMetrics(accums) - } - - test("existing values in input accums") { - // set input accum before passing it into TaskMetrics - val accums = InternalAccumulator.createAll() - val inAccum = accums.find(_.name === Some(input.RECORDS_READ)) - assert(inAccum.isDefined) - inAccum.get.asInstanceOf[Accumulator[Long]] += 10L - val tm = new TaskMetrics(accums) - } - - test("existing values in output accums") { - // set output accum before passing it into TaskMetrics - val accums = InternalAccumulator.createAll() - val outAccum = accums.find(_.name === Some(output.RECORDS_WRITTEN)) - assert(outAccum.isDefined) - outAccum.get.asInstanceOf[Accumulator[Long]] += 10L - val tm4 = new TaskMetrics(accums) + assert(newUpdates.size === tm.internalAccums.size + 4) } test("from accumulator updates") { - val accumUpdates1 = InternalAccumulator.createAll().map { a => + val accumUpdates1 = TaskMetrics.empty.internalAccums.map { a => AccumulableInfo(a.id, a.name, Some(3L), None, a.isInternal, a.countFailedValues) } val metrics1 = TaskMetrics.fromAccumulatorUpdates(accumUpdates1) @@ -412,29 +263,6 @@ class TaskMetricsSuite extends SparkFunSuite { private[spark] object TaskMetricsSuite extends Assertions { - /** - * Assert that the following three things are equal to `value`: - * (1) TaskMetrics value - * (2) TaskMetrics accumulator update value - * (3) Original accumulator value - */ - def assertValueEquals( - tm: TaskMetrics, - tmValue: TaskMetrics => Any, - accums: Seq[Accumulator[_]], - metricName: String, - value: Any, - assertEquals: (Any, Any) => Unit = (x: Any, y: Any) => assert(x === y)): Unit = { - assertEquals(tmValue(tm), value) - val accum = accums.find(_.name == Some(metricName)) - assert(accum.isDefined) - assertEquals(accum.get.value, value) - val accumUpdate = tm.accumulatorUpdates().find(_.name == Some(metricName)) - assert(accumUpdate.isDefined) - assert(accumUpdate.get.value === None) - assertEquals(accumUpdate.get.update, Some(value)) - } - /** * Assert that two lists of accumulator updates are equal. * Note: this does NOT check accumulator ID equality. @@ -458,5 +286,4 @@ private[spark] object TaskMetricsSuite extends Assertions { * info as an accumulator update. */ def makeInfo(a: Accumulable[_, _]): AccumulableInfo = a.toInfo(Some(a.value), None) - } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index fd96fb04f8b29..b76c0a4bd1dde 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -1144,7 +1144,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou // SPARK-9809 -- this stage is submitted without a task for each partition (because some of // the shuffle map output is still available from stage 0); make sure we've still got internal // accumulators setup - assert(scheduler.stageIdToStage(2).latestInfo.internalAccumulators.nonEmpty) + assert(scheduler.stageIdToStage(2).latestInfo.taskMetrics != null) completeShuffleMapStageSuccessfully(2, 0, 2) completeNextResultStageWithSuccess(3, 1, idx => idx + 1234) assert(results === Map(0 -> 1234, 1 -> 1235)) @@ -2010,7 +2010,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou extraAccumUpdates: Seq[AccumulableInfo] = Seq.empty[AccumulableInfo], taskInfo: TaskInfo = createFakeTaskInfo()): CompletionEvent = { val accumUpdates = reason match { - case Success => task.initialAccumulators.map { a => a.toInfo(Some(a.zero), None) } + case Success => task.metrics.accumulatorUpdates() case ef: ExceptionFailure => ef.accumUpdates case _ => Seq.empty[AccumulableInfo] } diff --git a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala index e3e6df6831def..4fe705b201ec8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala @@ -17,14 +17,11 @@ package org.apache.spark.scheduler -import java.util.Properties - import org.apache.spark.TaskContext class FakeTask( stageId: Int, - prefLocs: Seq[TaskLocation] = Nil) - extends Task[Int](stageId, 0, 0, Seq.empty, new Properties) { + prefLocs: Seq[TaskLocation] = Nil) extends Task[Int](stageId, 0, 0) { override def runTask(context: TaskContext): Int = 0 override def preferredLocations: Seq[TaskLocation] = prefLocs } diff --git a/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala index 76a7087645961..255be6f46b06b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala @@ -18,7 +18,6 @@ package org.apache.spark.scheduler import java.io.{IOException, ObjectInputStream, ObjectOutputStream} -import java.util.Properties import org.apache.spark.TaskContext @@ -26,7 +25,7 @@ import org.apache.spark.TaskContext * A Task implementation that fails to serialize. */ private[spark] class NotSerializableFakeTask(myId: Int, stageId: Int) - extends Task[Array[Byte]](stageId, 0, 0, Seq.empty, new Properties) { + extends Task[Array[Byte]](stageId, 0, 0) { override def runTask(context: TaskContext): Array[Byte] = Array.empty[Byte] override def preferredLocations: Seq[TaskLocation] = Seq[TaskLocation]() diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index 86911d2211a3a..bda4c996b27df 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -24,7 +24,7 @@ import org.mockito.Mockito._ import org.scalatest.BeforeAndAfter import org.apache.spark._ -import org.apache.spark.executor.{Executor, TaskMetricsSuite} +import org.apache.spark.executor.{Executor, TaskMetrics, TaskMetricsSuite} import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.metrics.source.JvmSource import org.apache.spark.network.util.JavaUtils @@ -62,7 +62,7 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark val func = (c: TaskContext, i: Iterator[String]) => i.next() val taskBinary = sc.broadcast(JavaUtils.bufferToArray(closureSerializer.serialize((rdd, func)))) val task = new ResultTask[String, String]( - 0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0, new Properties) + 0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0, new Properties, new TaskMetrics) intercept[RuntimeException] { task.run(0, 0, null) } @@ -83,7 +83,7 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark val func = (c: TaskContext, i: Iterator[String]) => i.next() val taskBinary = sc.broadcast(JavaUtils.bufferToArray(closureSerializer.serialize((rdd, func)))) val task = new ResultTask[String, String]( - 0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0, new Properties) + 0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0, new Properties, new TaskMetrics) intercept[RuntimeException] { task.run(0, 0, null) } @@ -171,26 +171,27 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark val param = AccumulatorParam.LongAccumulatorParam val acc1 = new Accumulator(0L, param, Some("x"), internal = false, countFailedValues = true) val acc2 = new Accumulator(0L, param, Some("y"), internal = false, countFailedValues = false) - val initialAccums = InternalAccumulator.createAll() // Create a dummy task. We won't end up running this; we just want to collect // accumulator updates from it. - val task = new Task[Int](0, 0, 0, Seq.empty[Accumulator[_]], new Properties) { + val taskMetrics = new TaskMetrics + val task = new Task[Int](0, 0, 0) { context = new TaskContextImpl(0, 0, 0L, 0, new TaskMemoryManager(SparkEnv.get.memoryManager, 0L), new Properties, SparkEnv.get.metricsSystem, - initialAccums) - context.taskMetrics.registerAccumulator(acc1) - context.taskMetrics.registerAccumulator(acc2) + taskMetrics) + taskMetrics.registerAccumulator(acc1) + taskMetrics.registerAccumulator(acc2) override def runTask(tc: TaskContext): Int = 0 } // First, simulate task success. This should give us all the accumulators. val accumUpdates1 = task.collectAccumulatorUpdates(taskFailed = false) - val accumUpdates2 = (initialAccums ++ Seq(acc1, acc2)).map(TaskMetricsSuite.makeInfo) + val accumUpdates2 = (taskMetrics.internalAccums ++ Seq(acc1, acc2)) + .map(TaskMetricsSuite.makeInfo) TaskMetricsSuite.assertUpdatesEquals(accumUpdates1, accumUpdates2) // Now, simulate task failures. This should give us only the accums that count failed values. val accumUpdates3 = task.collectAccumulatorUpdates(taskFailed = true) - val accumUpdates4 = (initialAccums ++ Seq(acc1)).map(TaskMetricsSuite.makeInfo) + val accumUpdates4 = (taskMetrics.internalAccums ++ Seq(acc1)).map(TaskMetricsSuite.makeInfo) TaskMetricsSuite.assertUpdatesEquals(accumUpdates3, accumUpdates4) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index ade8e84d848f0..ecf4b76da5586 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -17,9 +17,8 @@ package org.apache.spark.scheduler -import java.util.{Properties, Random} +import java.util.Random -import scala.collection.Map import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -138,7 +137,8 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex /** * A Task implementation that results in a large serialized task. */ -class LargeTask(stageId: Int) extends Task[Array[Byte]](stageId, 0, 0, Seq.empty, new Properties) { +class LargeTask(stageId: Int) extends Task[Array[Byte]](stageId, 0, 0) { + val randomBuffer = new Array[Byte](TaskSetManager.TASK_SIZE_TO_WARN_KB * 1024) val random = new Random(0) random.nextBytes(randomBuffer) @@ -166,7 +166,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskSet = FakeTask.createTaskSet(1) val clock = new ManualClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) - val accumUpdates = taskSet.tasks.head.initialAccumulators.map { a => a.toInfo(Some(0L), None) } + val accumUpdates = + taskSet.tasks.head.metrics.internalAccums.map { a => a.toInfo(Some(0L), None) } // Offer a host with NO_PREF as the constraint, // we should get a nopref task immediately since that's what we only have @@ -185,7 +186,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskSet = FakeTask.createTaskSet(3) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) val accumUpdatesByTask: Array[Seq[AccumulableInfo]] = taskSet.tasks.map { task => - task.initialAccumulators.map { a => a.toInfo(Some(0L), None) } + task.metrics.internalAccums.map { a => a.toInfo(Some(0L), None) } } // First three offers should all find tasks diff --git a/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala b/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala index 88817dccf3497..d223af1496a4b 100644 --- a/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/api/v1/AllStagesResourceSuite.scala @@ -38,7 +38,7 @@ class AllStagesResourceSuite extends SparkFunSuite { stageUiData.taskData = tasks val status = StageStatus.ACTIVE val stageInfo = new StageInfo( - 1, 1, "stage 1", 10, Seq.empty, Seq.empty, "details abc", Seq.empty) + 1, 1, "stage 1", 10, Seq.empty, Seq.empty, "details abc") val stageData = AllStagesResource.stageUiToStageData(status, stageInfo, stageUiData, false) stageData.firstTaskLaunchedTime diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 85c877e3ddfb9..221124829fc54 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -269,9 +269,7 @@ class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with val execId = "exe-1" def makeTaskMetrics(base: Int): TaskMetrics = { - val accums = InternalAccumulator.createAll() - accums.foreach(Accumulators.register) - val taskMetrics = new TaskMetrics(accums) + val taskMetrics = new TaskMetrics val shuffleReadMetrics = taskMetrics.createTempShuffleReadMetrics() val shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics val inputMetrics = taskMetrics.inputMetrics diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 7b15f5855886d..65b0a97e4dc12 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -656,6 +656,10 @@ object MimaExcludes { // [SPARK-14407] Hides HadoopFsRelation related data source API into execution package ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.OutputWriter"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.OutputWriterFactory") + ) ++ Seq( + // SPARK-14704: Create accumulators in TaskMetrics + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.InputMetrics.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.executor.OutputMetrics.this") ) case v if v.startsWith("1.6") => Seq( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala index 01687877eeed6..53105e0b24959 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala @@ -21,6 +21,7 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream, File} import java.util.Properties import org.apache.spark._ +import org.apache.spark.executor.TaskMetrics import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.rdd.RDD import org.apache.spark.sql.Row @@ -113,8 +114,7 @@ class UnsafeRowSerializerSuite extends SparkFunSuite with LocalSparkContext { (i, converter(Row(i))) } val taskMemoryManager = new TaskMemoryManager(sc.env.memoryManager, 0) - val taskContext = new TaskContextImpl( - 0, 0, 0, 0, taskMemoryManager, new Properties, null, InternalAccumulator.createAll(sc)) + val taskContext = new TaskContextImpl(0, 0, 0, 0, taskMemoryManager, new Properties, null) val sorter = new ExternalSorter[Int, UnsafeRow, UnsafeRow]( taskContext, From 856bc465d53ccfdfda75c82c85d7f318a5158088 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 19 Apr 2016 21:53:19 -0700 Subject: [PATCH 47/69] [SPARK-14600] [SQL] Push predicates through Expand ## What changes were proposed in this pull request? https://issues.apache.org/jira/browse/SPARK-14600 This PR makes `Expand.output` have different attributes from the grouping attributes produced by the underlying `Project`, as they have different meaning, so that we can safely push down filter through `Expand` ## How was this patch tested? existing tests. Author: Wenchen Fan Closes #12496 from cloud-fan/expand. --- .../spark/sql/catalyst/analysis/Analyzer.scala | 12 ++++++------ .../spark/sql/catalyst/optimizer/Optimizer.scala | 2 -- .../catalyst/plans/logical/basicOperators.scala | 5 ++++- .../catalyst/optimizer/FilterPushdownSuite.scala | 15 +++++++++++++++ .../org/apache/spark/sql/hive/SQLBuilder.scala | 14 +++++++++----- 5 files changed, 34 insertions(+), 14 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 236476900a519..8595762988b4b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -296,10 +296,13 @@ class Analyzer( val nonNullBitmask = x.bitmasks.reduce(_ & _) - val groupByAttributes = groupByAliases.zipWithIndex.map { case (a, idx) => + val expandedAttributes = groupByAliases.zipWithIndex.map { case (a, idx) => a.toAttribute.withNullability((nonNullBitmask & 1 << idx) == 0) } + val expand = Expand(x.bitmasks, groupByAliases, expandedAttributes, gid, x.child) + val groupingAttrs = expand.output.drop(x.child.output.length) + val aggregations: Seq[NamedExpression] = x.aggregations.map { case expr => // collect all the found AggregateExpression, so we can check an expression is part of // any AggregateExpression or not. @@ -321,15 +324,12 @@ class Analyzer( if (index == -1) { e } else { - groupByAttributes(index) + groupingAttrs(index) } }.asInstanceOf[NamedExpression] } - Aggregate( - groupByAttributes :+ gid, - aggregations, - Expand(x.bitmasks, groupByAliases, groupByAttributes, gid, x.child)) + Aggregate(groupingAttrs, aggregations, expand) case f @ Filter(cond, child) if hasGroupingFunction(cond) => val groupingExprs = findGroupingExprs(child) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index ecc2d773e7753..e6d554565d442 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -1020,8 +1020,6 @@ object PushDownPredicate extends Rule[LogicalPlan] with PredicateHelper { case filter @ Filter(_, f: Filter) => filter // should not push predicates through sample, or will generate different results. case filter @ Filter(_, s: Sample) => filter - // TODO: push predicates through expand - case filter @ Filter(_, e: Expand) => filter case filter @ Filter(condition, u: UnaryNode) if u.expressions.forall(_.deterministic) => pushDownPredicate(filter, u.child) { predicate => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index d4fc9e4da944a..a445ce694750a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -516,7 +516,10 @@ private[sql] object Expand { // groupingId is the last output, here we use the bit mask as the concrete value for it. } :+ Literal.create(bitmask, IntegerType) } - val output = child.output ++ groupByAttrs :+ gid + + // the `groupByAttrs` has different meaning in `Expand.output`, it could be the original + // grouping expression or null, so here we create new instance of it. + val output = child.output ++ groupByAttrs.map(_.newInstance) :+ gid Expand(projections, output, Project(child.output ++ groupByAliases, child)) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index df7529d83f7c8..9174b4e649a6e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -743,4 +743,19 @@ class FilterPushdownSuite extends PlanTest { comparePlans(optimized, correctAnswer) } + + test("expand") { + val agg = testRelation + .groupBy(Cube(Seq('a, 'b)))('a, 'b, sum('c)) + .analyze + .asInstanceOf[Aggregate] + + val a = agg.output(0) + val b = agg.output(1) + + val query = agg.where(a > 1 && b > 2) + val optimized = Optimize.execute(query) + val correctedAnswer = agg.copy(child = agg.child.where(a > 1 && b > 2)).analyze + comparePlans(optimized, correctedAnswer) + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala index e54358e657690..2d44813f0eac5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala @@ -288,8 +288,9 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends Loggi private def isGroupingSet(a: Aggregate, e: Expand, p: Project): Boolean = { assert(a.child == e && e.child == p) - a.groupingExpressions.forall(_.isInstanceOf[Attribute]) && - sameOutput(e.output, p.child.output ++ a.groupingExpressions.map(_.asInstanceOf[Attribute])) + a.groupingExpressions.forall(_.isInstanceOf[Attribute]) && sameOutput( + e.output.drop(p.child.output.length), + a.groupingExpressions.map(_.asInstanceOf[Attribute])) } private def groupingSetToSQL( @@ -303,25 +304,28 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends Loggi val numOriginalOutput = project.child.output.length // Assumption: Aggregate's groupingExpressions is composed of - // 1) the attributes of aliased group by expressions + // 1) the grouping attributes // 2) gid, which is always the last one val groupByAttributes = agg.groupingExpressions.dropRight(1).map(_.asInstanceOf[Attribute]) // Assumption: Project's projectList is composed of // 1) the original output (Project's child.output), // 2) the aliased group by expressions. + val expandedAttributes = project.output.drop(numOriginalOutput) val groupByExprs = project.projectList.drop(numOriginalOutput).map(_.asInstanceOf[Alias].child) val groupingSQL = groupByExprs.map(_.sql).mkString(", ") // a map from group by attributes to the original group by expressions. val groupByAttrMap = AttributeMap(groupByAttributes.zip(groupByExprs)) + // a map from expanded attributes to the original group by expressions. + val expandedAttrMap = AttributeMap(expandedAttributes.zip(groupByExprs)) val groupingSet: Seq[Seq[Expression]] = expand.projections.map { project => // Assumption: expand.projections is composed of // 1) the original output (Project's child.output), - // 2) group by attributes(or null literal) + // 2) expanded attributes(or null literal) // 3) gid, which is always the last one in each project in Expand project.drop(numOriginalOutput).dropRight(1).collect { - case attr: Attribute if groupByAttrMap.contains(attr) => groupByAttrMap(attr) + case attr: Attribute if expandedAttrMap.contains(attr) => expandedAttrMap(attr) } } val groupingSetSQL = "GROUPING SETS(" + From 6f1ec1f2670cd55bc852a810ca9d5c6a2651a9f2 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 19 Apr 2016 21:55:29 -0700 Subject: [PATCH 48/69] [MINOR] [SQL] Re-enable `explode()` and `json_tuple()` testcases in ExpressionToSQLSuite ## What changes were proposed in this pull request? Since [SPARK-12719: SQL Generation supports for generators](https://issues.apache.org/jira/browse/SPARK-12719) was resolved, this PR enables the related testcases: `explode()` and `json_tuple()`. ## How was this patch tested? Pass the Jenkins tests (with re-enabled test cases). Author: Dongjoon Hyun Closes #12329 from dongjoon-hyun/minor_enable_testcases. --- .../org/apache/spark/sql/hive/ExpressionToSQLSuite.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ExpressionToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ExpressionToSQLSuite.scala index bf85d71c66759..4d75becdb01d5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ExpressionToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ExpressionToSQLSuite.scala @@ -92,8 +92,7 @@ class ExpressionToSQLSuite extends SQLBuilderTest with SQLTestUtils { checkSqlGeneration("SELECT abs(15), abs(-15)") checkSqlGeneration("SELECT array(1,2,3)") checkSqlGeneration("SELECT coalesce(null, 1, 2)") - // wait for resolution of JIRA SPARK-12719 SQL Generation for Generators - // checkSqlGeneration("SELECT explode(array(1,2,3))") + checkSqlGeneration("SELECT explode(array(1,2,3))") checkSqlGeneration("SELECT greatest(1,null,3)") checkSqlGeneration("SELECT if(1==2, 'yes', 'no')") checkSqlGeneration("SELECT isnan(15), isnan('invalid')") @@ -200,8 +199,7 @@ class ExpressionToSQLSuite extends SQLBuilderTest with SQLTestUtils { checkSqlGeneration("SELECT locate('is', 'This is a test', 3)") checkSqlGeneration("SELECT lpad('SparkSql', 16, 'Learning')") checkSqlGeneration("SELECT ltrim(' SparkSql ')") - // wait for resolution of JIRA SPARK-12719 SQL Generation for Generators - // checkSqlGeneration("SELECT json_tuple('{\"f1\": \"value1\", \"f2\": \"value2\"}','f1')") + checkSqlGeneration("SELECT json_tuple('{\"f1\": \"value1\", \"f2\": \"value2\"}','f1')") checkSqlGeneration("SELECT printf('aa%d%s', 123, 'cc')") checkSqlGeneration("SELECT regexp_extract('100-200', '(\\d+)-(\\d+)', 1)") checkSqlGeneration("SELECT regexp_replace('100-200', '(\\d+)', 'num')") From 14869ae64eb27830179d4954a5dc3e0a1e1330b4 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 19 Apr 2016 22:28:11 -0700 Subject: [PATCH 49/69] [SPARK-14639] [PYTHON] [R] Add `bround` function in Python/R. ## What changes were proposed in this pull request? This issue aims to expose Scala `bround` function in Python/R API. `bround` function is implemented in SPARK-14614 by extending current `round` function. We used the following semantics from Hive. ```java public static double bround(double input, int scale) { if (Double.isNaN(input) || Double.isInfinite(input)) { return input; } return BigDecimal.valueOf(input).setScale(scale, RoundingMode.HALF_EVEN).doubleValue(); } ``` After this PR, `pyspark` and `sparkR` also support `bround` function. **PySpark** ```python >>> from pyspark.sql.functions import bround >>> sqlContext.createDataFrame([(2.5,)], ['a']).select(bround('a', 0).alias('r')).collect() [Row(r=2.0)] ``` **SparkR** ```r > df = createDataFrame(sqlContext, data.frame(x = c(2.5, 3.5))) > head(collect(select(df, bround(df$x, 0)))) bround(x, 0) 1 2 2 4 ``` ## How was this patch tested? Pass the Jenkins tests (including new testcases). Author: Dongjoon Hyun Closes #12509 from dongjoon-hyun/SPARK-14639. --- R/pkg/NAMESPACE | 1 + R/pkg/R/functions.R | 22 +++++++++++++++++++++- R/pkg/R/generics.R | 4 ++++ R/pkg/inst/tests/testthat/test_sparkSQL.R | 5 +++++ python/pyspark/sql/functions.py | 19 ++++++++++++++++--- 5 files changed, 47 insertions(+), 4 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 10b9d16279308..667fff7192b59 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -126,6 +126,7 @@ exportMethods("%in%", "between", "bin", "bitwiseNOT", + "bround", "cast", "cbrt", "ceil", diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index db877b2d63d30..54234b0455eab 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -994,7 +994,7 @@ setMethod("rint", #' round #' -#' Returns the value of the column `e` rounded to 0 decimal places. +#' Returns the value of the column `e` rounded to 0 decimal places using HALF_UP rounding mode. #' #' @rdname round #' @name round @@ -1008,6 +1008,26 @@ setMethod("round", column(jc) }) +#' bround +#' +#' Returns the value of the column `e` rounded to `scale` decimal places using HALF_EVEN rounding +#' mode if `scale` >= 0 or at integral part when `scale` < 0. +#' Also known as Gaussian rounding or bankers' rounding that rounds to the nearest even number. +#' bround(2.5, 0) = 2, bround(3.5, 0) = 4. +#' +#' @rdname bround +#' @name bround +#' @family math_funcs +#' @export +#' @examples \dontrun{bround(df$c, 0)} +setMethod("bround", + signature(x = "Column"), + function(x, scale = 0) { + jc <- callJStatic("org.apache.spark.sql.functions", "bround", x@jc, as.integer(scale)) + column(jc) + }) + + #' rtrim #' #' Trim the spaces from right end for the specified string value. diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index a71be55bcae81..6b67258d77e6c 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -760,6 +760,10 @@ setGeneric("bin", function(x) { standardGeneric("bin") }) #' @export setGeneric("bitwiseNOT", function(x) { standardGeneric("bitwiseNOT") }) +#' @rdname bround +#' @export +setGeneric("bround", function(x, ...) { standardGeneric("bround") }) + #' @rdname cbrt #' @export setGeneric("cbrt", function(x) { standardGeneric("cbrt") }) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 2f65484fcbdd8..b923ccf6bb1ae 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1087,6 +1087,11 @@ test_that("column functions", { expect_equal(collect(select(df, last(df$age, TRUE)))[[1]], 19) expect_equal(collect(select(df, last("age")))[[1]], 19) expect_equal(collect(select(df, last("age", TRUE)))[[1]], 19) + + # Test bround() + df <- createDataFrame(sqlContext, data.frame(x = c(2.5, 3.5))) + expect_equal(collect(select(df, bround(df$x, 0)))[[1]][1], 2) + expect_equal(collect(select(df, bround(df$x, 0)))[[1]][2], 4) }) test_that("column binary mathfunctions", { diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 5017ab5b3646d..dac842c0ce8c0 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -467,16 +467,29 @@ def randn(seed=None): @since(1.5) def round(col, scale=0): """ - Round the value of `e` to `scale` decimal places if `scale` >= 0 + Round the given value to `scale` decimal places using HALF_UP rounding mode if `scale` >= 0 or at integral part when `scale` < 0. - >>> sqlContext.createDataFrame([(2.546,)], ['a']).select(round('a', 1).alias('r')).collect() - [Row(r=2.5)] + >>> sqlContext.createDataFrame([(2.5,)], ['a']).select(round('a', 0).alias('r')).collect() + [Row(r=3.0)] """ sc = SparkContext._active_spark_context return Column(sc._jvm.functions.round(_to_java_column(col), scale)) +@since(2.0) +def bround(col, scale=0): + """ + Round the given value to `scale` decimal places using HALF_EVEN rounding mode if `scale` >= 0 + or at integral part when `scale` < 0. + + >>> sqlContext.createDataFrame([(2.5,)], ['a']).select(bround('a', 0).alias('r')).collect() + [Row(r=2.0)] + """ + sc = SparkContext._active_spark_context + return Column(sc._jvm.functions.bround(_to_java_column(col), scale)) + + @since(1.5) def shiftLeft(col, numBits): """Shift the given value numBits left. From 7abe9a6578a99f4df50040d5cfe083c389c7b97f Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 20 Apr 2016 00:44:02 -0700 Subject: [PATCH 50/69] [SPARK-9013][SQL] generate MutableProjection directly instead of return a function `MutableProjection` is not thread-safe and we won't use it in multiple threads. I think the reason that we return `() => MutableProjection` is not about thread safety, but to save the costs of generating code when we need same but individual mutable projections. However, I only found one place that use this [feature](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala#L122-L123), and comparing to the troubles it brings, I think we should generate `MutableProjection` directly instead of return a function. Author: Wenchen Fan Closes #7373 from cloud-fan/project. --- .../codegen/GenerateMutableProjection.scala | 12 +++++------ .../expressions/CodeGenerationSuite.scala | 4 ++-- .../expressions/ExpressionEvalHelper.scala | 6 +++--- .../expressions/MathFunctionsSuite.scala | 2 +- .../CodegenExpressionCachingSuite.scala | 6 +++--- .../codegen/GeneratedProjectionSuite.scala | 2 +- .../spark/sql/execution/SparkPlan.scala | 4 +--- .../apache/spark/sql/execution/Window.scala | 20 +++++++++---------- .../aggregate/AggregationIterator.scala | 8 ++++---- .../SortBasedAggregationIterator.scala | 2 +- .../aggregate/TungstenAggregate.scala | 2 +- .../TungstenAggregationIterator.scala | 2 +- .../spark/sql/execution/aggregate/udaf.scala | 2 +- .../python/BatchPythonEvaluation.scala | 2 +- 14 files changed, 35 insertions(+), 39 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala index 7f840890f8ae5..f143b40443836 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala @@ -29,7 +29,7 @@ abstract class BaseMutableProjection extends MutableProjection * It exposes a `target` method, which is used to set the row that will be updated. * The internal [[MutableRow]] object created internally is used only when `target` is not used. */ -object GenerateMutableProjection extends CodeGenerator[Seq[Expression], () => MutableProjection] { +object GenerateMutableProjection extends CodeGenerator[Seq[Expression], MutableProjection] { protected def canonicalize(in: Seq[Expression]): Seq[Expression] = in.map(ExpressionCanonicalizer.execute) @@ -40,17 +40,17 @@ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], () => Mu def generate( expressions: Seq[Expression], inputSchema: Seq[Attribute], - useSubexprElimination: Boolean): (() => MutableProjection) = { + useSubexprElimination: Boolean): MutableProjection = { create(canonicalize(bind(expressions, inputSchema)), useSubexprElimination) } - protected def create(expressions: Seq[Expression]): (() => MutableProjection) = { + protected def create(expressions: Seq[Expression]): MutableProjection = { create(expressions, false) } private def create( expressions: Seq[Expression], - useSubexprElimination: Boolean): (() => MutableProjection) = { + useSubexprElimination: Boolean): MutableProjection = { val ctx = newCodeGenContext() val (validExpr, index) = expressions.zipWithIndex.filter { case (NoOp, _) => false @@ -136,8 +136,6 @@ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], () => Mu logDebug(s"code for ${expressions.mkString(",")}:\n${CodeFormatter.format(code)}") val c = CodeGenerator.compile(code) - () => { - c.generate(ctx.references.toArray).asInstanceOf[MutableProjection] - } + c.generate(ctx.references.toArray).asInstanceOf[MutableProjection] } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala index 94e676ded601b..b682e7d2b1d8e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala @@ -50,7 +50,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { test("SPARK-8443: split wide projections into blocks due to JVM code size limit") { val length = 5000 val expressions = List.fill(length)(EqualTo(Literal(1), Literal(1))) - val plan = GenerateMutableProjection.generate(expressions)() + val plan = GenerateMutableProjection.generate(expressions) val actual = plan(new GenericMutableRow(length)).toSeq(expressions.map(_.dataType)) val expected = Seq.fill(length)(true) @@ -73,7 +73,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { val expression = CaseWhen((1 to cases).map(generateCase(_))) - val plan = GenerateMutableProjection.generate(Seq(expression))() + val plan = GenerateMutableProjection.generate(Seq(expression)) val input = new GenericMutableRow(Array[Any](UTF8String.fromString(s"${clauses}:${cases}"))) val actual = plan(input).toSeq(Seq(expression.dataType)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala index faa90fb1c5e35..8a9617cfbf5df 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala @@ -110,7 +110,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks { inputRow: InternalRow = EmptyRow): Unit = { val plan = generateProject( - GenerateMutableProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil)(), + GenerateMutableProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil), expression) val actual = plan(inputRow).get(0, expression.dataType) @@ -166,7 +166,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks { checkEvaluationWithOptimization(expression, expected) var plan = generateProject( - GenerateMutableProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil)(), + GenerateMutableProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil), expression) var actual = plan(inputRow).get(0, expression.dataType) assert(checkResult(actual, expected)) @@ -259,7 +259,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks { } val plan = generateProject( - GenerateMutableProjection.generate(Alias(expr, s"Optimized($expr)")() :: Nil)(), + GenerateMutableProjection.generate(Alias(expr, s"Optimized($expr)")() :: Nil), expr) val codegen = plan(inputRow).get(0, expr.dataType) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala index 1e5b657f1fb98..f88c9e8df16d0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala @@ -138,7 +138,7 @@ class MathFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { inputRow: InternalRow = EmptyRow): Unit = { val plan = generateProject( - GenerateMutableProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil)(), + GenerateMutableProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil), expression) val actual = plan(inputRow).get(0, expression.dataType) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenExpressionCachingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenExpressionCachingSuite.scala index c9616cdb26c20..06dc3bd33b90e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenExpressionCachingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenExpressionCachingSuite.scala @@ -36,7 +36,7 @@ class CodegenExpressionCachingSuite extends SparkFunSuite { test("GenerateMutableProjection should initialize expressions") { val expr = And(NondeterministicExpression(), NondeterministicExpression()) - val instance = GenerateMutableProjection.generate(Seq(expr))() + val instance = GenerateMutableProjection.generate(Seq(expr)) assert(instance.apply(null).getBoolean(0) === false) } @@ -60,12 +60,12 @@ class CodegenExpressionCachingSuite extends SparkFunSuite { test("GenerateMutableProjection should not share expression instances") { val expr1 = MutableExpression() - val instance1 = GenerateMutableProjection.generate(Seq(expr1))() + val instance1 = GenerateMutableProjection.generate(Seq(expr1)) assert(instance1.apply(null).getBoolean(0) === false) val expr2 = MutableExpression() expr2.mutableState = true - val instance2 = GenerateMutableProjection.generate(Seq(expr2))() + val instance2 = GenerateMutableProjection.generate(Seq(expr2)) assert(instance1.apply(null).getBoolean(0) === false) assert(instance2.apply(null).getBoolean(0) === true) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala index e2a8eb8ee1d34..b69b74b4240bd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala @@ -76,7 +76,7 @@ class GeneratedProjectionSuite extends SparkFunSuite { val exprs = nestedSchema.fields.zipWithIndex.map { case (f, i) => BoundReference(i, f.dataType, true) } - val mutableProj = GenerateMutableProjection.generate(exprs)() + val mutableProj = GenerateMutableProjection.generate(exprs) val row1 = mutableProj(result) assert(result === row1) val row2 = mutableProj(result) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 415cd4d84a23f..b64352a9e0dc2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -352,12 +352,10 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ } } - private[this] def isTesting: Boolean = sys.props.contains("spark.testing") - protected def newMutableProjection( expressions: Seq[Expression], inputSchema: Seq[Attribute], - useSubexprElimination: Boolean = false): () => MutableProjection = { + useSubexprElimination: Boolean = false): MutableProjection = { log.debug(s"Creating MutableProj: $expressions, inputSchema: $inputSchema") GenerateMutableProjection.generate(expressions, inputSchema, useSubexprElimination) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala index 8e9214fa258b2..85ce388de0aa4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala @@ -120,14 +120,14 @@ case class Window( val (exprs, current, bound) = if (offset == 0) { // Use the entire order expression when the offset is 0. val exprs = orderSpec.map(_.child) - val projection = newMutableProjection(exprs, child.output) - (orderSpec, projection(), projection()) + val buildProjection = () => newMutableProjection(exprs, child.output) + (orderSpec, buildProjection(), buildProjection()) } else if (orderSpec.size == 1) { // Use only the first order expression when the offset is non-null. val sortExpr = orderSpec.head val expr = sortExpr.child // Create the projection which returns the current 'value'. - val current = newMutableProjection(expr :: Nil, child.output)() + val current = newMutableProjection(expr :: Nil, child.output) // Flip the sign of the offset when processing the order is descending val boundOffset = sortExpr.direction match { case Descending => -offset @@ -135,7 +135,7 @@ case class Window( } // Create the projection which returns the current 'value' modified by adding the offset. val boundExpr = Add(expr, Cast(Literal.create(boundOffset, IntegerType), expr.dataType)) - val bound = newMutableProjection(boundExpr :: Nil, child.output)() + val bound = newMutableProjection(boundExpr :: Nil, child.output) (sortExpr :: Nil, current, bound) } else { sys.error("Non-Zero range offsets are not supported for windows " + @@ -564,7 +564,7 @@ private[execution] final class OffsetWindowFunctionFrame( ordinal: Int, expressions: Array[Expression], inputSchema: Seq[Attribute], - newMutableProjection: (Seq[Expression], Seq[Attribute]) => () => MutableProjection, + newMutableProjection: (Seq[Expression], Seq[Attribute]) => MutableProjection, offset: Int) extends WindowFunctionFrame { /** Rows of the partition currently being processed. */ @@ -604,7 +604,7 @@ private[execution] final class OffsetWindowFunctionFrame( } // Create the projection. - newMutableProjection(boundExpressions, Nil)().target(target) + newMutableProjection(boundExpressions, Nil).target(target) } override def prepare(rows: RowBuffer): Unit = { @@ -886,7 +886,7 @@ private[execution] object AggregateProcessor { functions: Array[Expression], ordinal: Int, inputAttributes: Seq[Attribute], - newMutableProjection: (Seq[Expression], Seq[Attribute]) => () => MutableProjection): + newMutableProjection: (Seq[Expression], Seq[Attribute]) => MutableProjection): AggregateProcessor = { val aggBufferAttributes = mutable.Buffer.empty[AttributeReference] val initialValues = mutable.Buffer.empty[Expression] @@ -938,13 +938,13 @@ private[execution] object AggregateProcessor { // Create the projections. val initialProjection = newMutableProjection( initialValues, - partitionSize.toSeq)() + partitionSize.toSeq) val updateProjection = newMutableProjection( updateExpressions, - aggBufferAttributes ++ inputAttributes)() + aggBufferAttributes ++ inputAttributes) val evaluateProjection = newMutableProjection( evaluateExpressions, - aggBufferAttributes)() + aggBufferAttributes) // Create the processor new AggregateProcessor( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala index 042c7319018be..81aacb437ba54 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala @@ -39,7 +39,7 @@ abstract class AggregationIterator( aggregateAttributes: Seq[Attribute], initialInputBufferOffset: Int, resultExpressions: Seq[NamedExpression], - newMutableProjection: (Seq[Expression], Seq[Attribute]) => (() => MutableProjection)) + newMutableProjection: (Seq[Expression], Seq[Attribute]) => MutableProjection) extends Iterator[UnsafeRow] with Logging { /////////////////////////////////////////////////////////////////////////// @@ -139,7 +139,7 @@ abstract class AggregationIterator( // no-op expressions which are ignored during projection code-generation. case i: ImperativeAggregate => Seq.fill(i.aggBufferAttributes.length)(NoOp) } - newMutableProjection(initExpressions, Nil)() + newMutableProjection(initExpressions, Nil) } // All imperative AggregateFunctions. @@ -175,7 +175,7 @@ abstract class AggregationIterator( // This projection is used to merge buffer values for all expression-based aggregates. val aggregationBufferSchema = functions.flatMap(_.aggBufferAttributes) val updateProjection = - newMutableProjection(mergeExpressions, aggregationBufferSchema ++ inputAttributes)() + newMutableProjection(mergeExpressions, aggregationBufferSchema ++ inputAttributes) (currentBuffer: MutableRow, row: InternalRow) => { // Process all expression-based aggregate functions. @@ -211,7 +211,7 @@ abstract class AggregationIterator( case agg: AggregateFunction => NoOp } val aggregateResult = new SpecificMutableRow(aggregateAttributes.map(_.dataType)) - val expressionAggEvalProjection = newMutableProjection(evalExpressions, bufferAttributes)() + val expressionAggEvalProjection = newMutableProjection(evalExpressions, bufferAttributes) expressionAggEvalProjection.target(aggregateResult) val resultProjection = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala index de1491d357405..c35d781d3ebf5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala @@ -34,7 +34,7 @@ class SortBasedAggregationIterator( aggregateAttributes: Seq[Attribute], initialInputBufferOffset: Int, resultExpressions: Seq[NamedExpression], - newMutableProjection: (Seq[Expression], Seq[Attribute]) => (() => MutableProjection), + newMutableProjection: (Seq[Expression], Seq[Attribute]) => MutableProjection, numOutputRows: LongSQLMetric) extends AggregationIterator( groupingExpressions, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala index 89977f9e086b8..d4cef8f310dac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala @@ -335,7 +335,7 @@ case class TungstenAggregate( val mergeProjection = newMutableProjection( mergeExpr, aggregateBufferAttributes ++ declFunctions.flatMap(_.inputAggBufferAttributes), - subexpressionEliminationEnabled)() + subexpressionEliminationEnabled) val joinedRow = new JoinedRow() var currentKey: UnsafeRow = null diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala index 09384a482d9fd..c3687266109f3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala @@ -82,7 +82,7 @@ class TungstenAggregationIterator( aggregateAttributes: Seq[Attribute], initialInputBufferOffset: Int, resultExpressions: Seq[NamedExpression], - newMutableProjection: (Seq[Expression], Seq[Attribute]) => (() => MutableProjection), + newMutableProjection: (Seq[Expression], Seq[Attribute]) => MutableProjection, originalInputAttributes: Seq[Attribute], inputIter: Iterator[InternalRow], testFallbackStartsAt: Option[(Int, Int)], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala index f5776e7b8d49a..4ceb710f4b2b4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala @@ -361,7 +361,7 @@ private[sql] case class ScalaUDAF( val inputAttributes = childrenSchema.toAttributes log.debug( s"Creating MutableProj: $children, inputSchema: $inputAttributes.") - GenerateMutableProjection.generate(children, inputAttributes)() + GenerateMutableProjection.generate(children, inputAttributes) } private[this] lazy val inputToScalaConverters: Any => Any = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchPythonEvaluation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchPythonEvaluation.scala index c9ab40a0a9abf..c49f173ad6dff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchPythonEvaluation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchPythonEvaluation.scala @@ -86,7 +86,7 @@ case class BatchPythonEvaluation(udfs: Seq[PythonUDF], output: Seq[Attribute], c } }.toArray }.toArray - val projection = newMutableProjection(allInputs, child.output)() + val projection = newMutableProjection(allInputs, child.output) val schema = StructType(dataTypes.map(dt => StructField("", dt))) val needConversion = dataTypes.exists(EvaluatePython.needConversionInPython) From a3451119d951949f24f3a4c5e33a5daea615dfed Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Wed, 20 Apr 2016 11:26:42 +0100 Subject: [PATCH 51/69] [SPARK-14679][UI] Fix UI DAG visualization OOM. ## What changes were proposed in this pull request? The DAG visualization can cause an OOM when generating the DOT file. This happens because clusters are not correctly deduped by a contains check because they use the default equals implementation. This adds a working equals implementation. ## How was this patch tested? This adds a test suite that checks the new equals implementation. Author: Ryan Blue Closes #12437 from rdblue/SPARK-14679-fix-ui-oom. --- .../spark/ui/scope/RDDOperationGraph.scala | 18 +++++++++ .../ui/scope/RDDOperationGraphSuite.scala | 37 +++++++++++++++++++ 2 files changed, 55 insertions(+) create mode 100644 core/src/test/scala/org/apache/spark/ui/scope/RDDOperationGraphSuite.scala diff --git a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala index bb6b663f1ead3..84ca750e1a96a 100644 --- a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala +++ b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala @@ -17,6 +17,8 @@ package org.apache.spark.ui.scope +import java.util.Objects + import scala.collection.mutable import scala.collection.mutable.{ListBuffer, StringBuilder} @@ -72,6 +74,22 @@ private[ui] class RDDOperationCluster(val id: String, private var _name: String) def getCachedNodes: Seq[RDDOperationNode] = { _childNodes.filter(_.cached) ++ _childClusters.flatMap(_.getCachedNodes) } + + def canEqual(other: Any): Boolean = other.isInstanceOf[RDDOperationCluster] + + override def equals(other: Any): Boolean = other match { + case that: RDDOperationCluster => + (that canEqual this) && + _childClusters == that._childClusters && + id == that.id && + _name == that._name + case _ => false + } + + override def hashCode(): Int = { + val state = Seq(_childClusters, id, _name) + state.map(Objects.hashCode).foldLeft(0)((a, b) => 31 * a + b) + } } private[ui] object RDDOperationGraph extends Logging { diff --git a/core/src/test/scala/org/apache/spark/ui/scope/RDDOperationGraphSuite.scala b/core/src/test/scala/org/apache/spark/ui/scope/RDDOperationGraphSuite.scala new file mode 100644 index 0000000000000..6ddcb5aba1678 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/ui/scope/RDDOperationGraphSuite.scala @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ui.scope + +import org.apache.spark.SparkFunSuite + +class RDDOperationGraphSuite extends SparkFunSuite { + test("Test simple cluster equals") { + // create a 2-cluster chain with a child + val c1 = new RDDOperationCluster("1", "Bender") + val c2 = new RDDOperationCluster("2", "Hal") + c1.attachChildCluster(c2) + c1.attachChildNode(new RDDOperationNode(3, "Marvin", false, "collect!")) + + // create an equal cluster, but without the child node + val c1copy = new RDDOperationCluster("1", "Bender") + val c2copy = new RDDOperationCluster("2", "Hal") + c1copy.attachChildCluster(c2copy) + + assert(c1 == c1copy) + } +} From 17db4bfeaa0074298db622db38a5b0459518c4a9 Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Wed, 20 Apr 2016 11:28:51 +0100 Subject: [PATCH 52/69] [SPARK-14687][CORE][SQL][MLLIB] Call path.getFileSystem(conf) instead of call FileSystem.get(conf) ## What changes were proposed in this pull request? - replaced `FileSystem.get(conf)` calls with `path.getFileSystem(conf)` ## How was this patch tested? N/A Author: Liwei Lin Closes #12450 from lw-lin/fix-fs-get. --- .../org/apache/spark/deploy/history/FsHistoryProvider.scala | 4 ++-- .../main/scala/org/apache/spark/ml/recommendation/ALS.scala | 3 ++- .../test/scala/org/apache/spark/ml/clustering/LDASuite.scala | 5 +++-- .../datasources/parquet/SpecificParquetRecordReaderBase.java | 2 +- .../spark/sql/execution/streaming/FileStreamSource.scala | 2 +- 5 files changed, 9 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index d5afb33c7118a..2bd4a46e16fc9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -353,7 +353,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) * the name of the file being compressed. */ def zipFileToStream(file: Path, entryName: String, outputStream: ZipOutputStream): Unit = { - val fs = FileSystem.get(hadoopConf) + val fs = file.getFileSystem(hadoopConf) val inputStream = fs.open(file, 1 * 1024 * 1024) // 1MB Buffer try { outputStream.putNextEntry(new ZipEntry(entryName)) @@ -372,7 +372,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) attempt.attemptId.isEmpty || attemptId.isEmpty || attempt.attemptId.get == attemptId.get }.foreach { attempt => val logPath = new Path(logDir, attempt.logPath) - zipFileToStream(new Path(logDir, attempt.logPath), attempt.logPath, zipStream) + zipFileToStream(logPath, attempt.logPath, zipStream) } } finally { zipStream.close() diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index 36dce015908eb..728b4d1598938 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -640,7 +640,8 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { val deletePreviousCheckpointFile: () => Unit = () => previousCheckpointFile.foreach { file => try { - FileSystem.get(sc.hadoopConfiguration).delete(new Path(file), true) + val checkpointFile = new Path(file) + checkpointFile.getFileSystem(sc.hadoopConfiguration).delete(checkpointFile, true) } catch { case e: IOException => logWarning(s"Cannot delete checkpoint file $file:", e) diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala index 17d6e9fc2ee76..6cb07aecb952c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala @@ -273,8 +273,9 @@ class LDASuite extends SparkFunSuite with MLlibTestSparkContext with DefaultRead // There should be 1 checkpoint remaining. assert(model.getCheckpointFiles.length === 1) - val fs = FileSystem.get(sqlContext.sparkContext.hadoopConfiguration) - assert(fs.exists(new Path(model.getCheckpointFiles.head))) + val checkpointFile = new Path(model.getCheckpointFiles.head) + val fs = checkpointFile.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) + assert(fs.exists(checkpointFile)) model.deleteCheckpointFiles() assert(model.getCheckpointFiles.isEmpty) } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java index 5c257bc260873..b224a868454a5 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java @@ -178,7 +178,7 @@ protected void initialize(String path, List columns) throws IOException config.set("spark.sql.parquet.writeLegacyFormat", "false"); this.file = new Path(path); - long length = FileSystem.get(config).getFileStatus(this.file).getLen(); + long length = this.file.getFileSystem(config).getFileStatus(this.file).getLen(); ParquetMetadata footer = readFooter(config, file, range(0, length)); List blocks = footer.getBlocks(); diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index 1b70055f346b3..6448cb6e902fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -39,7 +39,7 @@ class FileStreamSource( providerName: String, dataFrameBuilder: Array[String] => DataFrame) extends Source with Logging { - private val fs = FileSystem.get(sqlContext.sparkContext.hadoopConfiguration) + private val fs = new Path(path).getFileSystem(sqlContext.sparkContext.hadoopConfiguration) private val metadataLog = new HDFSMetadataLog[Seq[String]](sqlContext, metadataPath) private var maxBatchId = metadataLog.getLatest().map(_._1).getOrElse(-1L) From ed9d80385486cd39a84a689ef467795262af919a Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Wed, 20 Apr 2016 11:45:08 +0100 Subject: [PATCH 53/69] [SPARK-14635][ML] Documentation and Examples for TF-IDF only refer to HashingTF ## What changes were proposed in this pull request? Currently, the docs for TF-IDF only refer to using HashingTF with IDF. However, CountVectorizer can also be used. We should probably amend the user guide and examples to show this. ## How was this patch tested? unit tests and doc generation Author: Yuhao Yang Closes #12454 from hhbyyh/tfdoc. --- docs/ml-features.md | 15 ++++++++++++--- .../spark/examples/ml/JavaTfIdfExample.java | 2 ++ examples/src/main/python/ml/tf_idf_example.py | 2 ++ .../apache/spark/examples/ml/TfIdfExample.scala | 2 ++ 4 files changed, 18 insertions(+), 3 deletions(-) diff --git a/docs/ml-features.md b/docs/ml-features.md index 876d21f49525a..11d5acbb10c30 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -22,10 +22,19 @@ This section covers algorithms for working with features, roughly divided into t [Term Frequency-Inverse Document Frequency (TF-IDF)](http://en.wikipedia.org/wiki/Tf%E2%80%93idf) is a common text pre-processing step. In Spark ML, TF-IDF is separate into two parts: TF (+hashing) and IDF. -**TF**: `HashingTF` is a `Transformer` which takes sets of terms and converts those sets into fixed-length feature vectors. In text processing, a "set of terms" might be a bag of words. -The algorithm combines Term Frequency (TF) counts with the [hashing trick](http://en.wikipedia.org/wiki/Feature_hashing) for dimensionality reduction. +**TF**: Both `HashingTF` and `CountVectorizer` can be used to generate the term frequency vectors. -**IDF**: `IDF` is an `Estimator` which fits on a dataset and produces an `IDFModel`. The `IDFModel` takes feature vectors (generally created from `HashingTF`) and scales each column. Intuitively, it down-weights columns which appear frequently in a corpus. +`HashingTF` is a `Transformer` which takes sets of terms and converts those sets into +fixed-length feature vectors. In text processing, a "set of terms" might be a bag of words. +The algorithm combines Term Frequency (TF) counts with the +[hashing trick](http://en.wikipedia.org/wiki/Feature_hashing) for dimensionality reduction. + +`CountVectorizer` converts text documents to vectors of term counts. Refer to [CountVectorizer +](ml-features.html#countvectorizer) for more details. + +**IDF**: `IDF` is an `Estimator` which is fit on a dataset and produces an `IDFModel`. The +`IDFModel` takes feature vectors (generally created from `HashingTF` or `CountVectorizer`) and scales each column. +Intuitively, it down-weights columns which appear frequently in a corpus. Please refer to the [MLlib user guide on TF-IDF](mllib-feature-extraction.html#tf-idf) for more details on Term Frequency and Inverse Document Frequency. diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java index 37a3d0d84dae2..107c835f2e01e 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java @@ -63,6 +63,8 @@ public static void main(String[] args) { .setOutputCol("rawFeatures") .setNumFeatures(numFeatures); Dataset featurizedData = hashingTF.transform(wordsData); + // alternatively, CountVectorizer can also be used to get term frequency vectors + IDF idf = new IDF().setInputCol("rawFeatures").setOutputCol("features"); IDFModel idfModel = idf.fit(featurizedData); Dataset rescaledData = idfModel.transform(featurizedData); diff --git a/examples/src/main/python/ml/tf_idf_example.py b/examples/src/main/python/ml/tf_idf_example.py index c92313378eec7..141324d458530 100644 --- a/examples/src/main/python/ml/tf_idf_example.py +++ b/examples/src/main/python/ml/tf_idf_example.py @@ -37,6 +37,8 @@ wordsData = tokenizer.transform(sentenceData) hashingTF = HashingTF(inputCol="words", outputCol="rawFeatures", numFeatures=20) featurizedData = hashingTF.transform(wordsData) + # alternatively, CountVectorizer can also be used to get term frequency vectors + idf = IDF(inputCol="rawFeatures", outputCol="features") idfModel = idf.fit(featurizedData) rescaledData = idfModel.transform(featurizedData) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala index 28115f939082e..396f073e6b322 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/TfIdfExample.scala @@ -43,6 +43,8 @@ object TfIdfExample { val hashingTF = new HashingTF() .setInputCol("words").setOutputCol("rawFeatures").setNumFeatures(20) val featurizedData = hashingTF.transform(wordsData) + // alternatively, CountVectorizer can also be used to get term frequency vectors + val idf = new IDF().setInputCol("rawFeatures").setOutputCol("features") val idfModel = idf.fit(featurizedData) val rescaledData = idfModel.transform(featurizedData) From 834277884fcdaab4758604272881ffb2369e25f0 Mon Sep 17 00:00:00 2001 From: Alex Bozarth Date: Wed, 20 Apr 2016 21:24:11 +0900 Subject: [PATCH 54/69] [SPARK-8171][WEB UI] Javascript based infinite scrolling for the log page Updated the log page by replacing the current pagination with a javascript-based infinite scroll solution Author: Alex Bozarth Closes #10910 from ajbozarth/spark8171. --- .../org/apache/spark/ui/static/log-view.js | 129 ++++++++++++++++++ .../org/apache/spark/ui/static/webui.css | 10 ++ .../spark/deploy/worker/ui/LogPage.scala | 75 +++++----- .../org/apache/spark/ui/JettyUtils.scala | 4 +- .../scala/org/apache/spark/ui/UIUtils.scala | 1 + 5 files changed, 175 insertions(+), 44 deletions(-) create mode 100644 core/src/main/resources/org/apache/spark/ui/static/log-view.js diff --git a/core/src/main/resources/org/apache/spark/ui/static/log-view.js b/core/src/main/resources/org/apache/spark/ui/static/log-view.js new file mode 100644 index 0000000000000..1782b4f209c09 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/log-view.js @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +var baseParams; + +var curLogLength; +var startByte; +var endByte; +var totalLogLength; + +var byteLength; + +function setLogScroll(oldHeight) { + var logContent = $(".log-content"); + logContent.scrollTop(logContent[0].scrollHeight - oldHeight); +} + +function tailLog() { + var logContent = $(".log-content"); + logContent.scrollTop(logContent[0].scrollHeight); +} + +function setLogData() { + $('#log-data').html("Showing " + curLogLength + " Bytes: " + startByte + + " - " + endByte + " of " + totalLogLength); +} + +function disableMoreButton() { + var moreBtn = $(".log-more-btn"); + moreBtn.attr("disabled", "disabled"); + moreBtn.html("Top of Log"); +} + +function noNewAlert() { + var alert = $(".no-new-alert"); + alert.css("display", "block"); + window.setTimeout(function () {alert.css("display", "none");}, 4000); +} + +function loadMore() { + var offset = Math.max(startByte - byteLength, 0); + var moreByteLength = Math.min(byteLength, startByte); + + $.ajax({ + type: "GET", + url: "/log" + baseParams + "&offset=" + offset + "&byteLength=" + moreByteLength, + success: function (data) { + var oldHeight = $(".log-content")[0].scrollHeight; + var newlineIndex = data.indexOf('\n'); + var dataInfo = data.substring(0, newlineIndex).match(/\d+/g); + var retStartByte = dataInfo[0]; + var retLogLength = dataInfo[2]; + + var cleanData = data.substring(newlineIndex + 1); + if (retStartByte == 0) { + disableMoreButton(); + } + $("pre", ".log-content").prepend(cleanData); + + curLogLength = curLogLength + (startByte - retStartByte); + startByte = retStartByte; + totalLogLength = retLogLength; + setLogScroll(oldHeight); + setLogData(); + } + }); +} + +function loadNew() { + $.ajax({ + type: "GET", + url: "/log" + baseParams + "&byteLength=0", + success: function (data) { + var dataInfo = data.substring(0, data.indexOf('\n')).match(/\d+/g); + var newDataLen = dataInfo[2] - totalLogLength; + if (newDataLen != 0) { + $.ajax({ + type: "GET", + url: "/log" + baseParams + "&byteLength=" + newDataLen, + success: function (data) { + var newlineIndex = data.indexOf('\n'); + var dataInfo = data.substring(0, newlineIndex).match(/\d+/g); + var retStartByte = dataInfo[0]; + var retEndByte = dataInfo[1]; + var retLogLength = dataInfo[2]; + + var cleanData = data.substring(newlineIndex + 1); + $("pre", ".log-content").append(cleanData); + + curLogLength = curLogLength + (retEndByte - retStartByte); + endByte = retEndByte; + totalLogLength = retLogLength; + tailLog(); + setLogData(); + } + }); + } else { + noNewAlert(); + } + } + }); +} + +function initLogPage(params, logLen, start, end, totLogLen, defaultLen) { + baseParams = params; + curLogLength = logLen; + startByte = start; + endByte = end; + totalLogLength = totLogLen; + byteLength = defaultLen; + tailLog(); + if (startByte == 0) { + disableMoreButton(); + } +} \ No newline at end of file diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index 47dd9162a1bfa..595e80ab5e3ad 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -237,3 +237,13 @@ a.expandbutton { color: #333; text-decoration: none; } + +.log-more-btn, .log-new-btn { + width: 100% +} + +.no-new-alert { + text-align: center; + margin: 0; + padding: 4px 0; +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index e75c0cec4acc7..3473c41b935fd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -20,7 +20,7 @@ package org.apache.spark.deploy.worker.ui import java.io.File import javax.servlet.http.HttpServletRequest -import scala.xml.Node +import scala.xml.{Node, Unparsed} import org.apache.spark.internal.Logging import org.apache.spark.ui.{UIUtils, WebUIPage} @@ -31,10 +31,9 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with private val worker = parent.worker private val workDir = new File(parent.workDir.toURI.normalize().getPath) private val supportedLogTypes = Set("stderr", "stdout") + private val defaultBytes = 100 * 1024 def renderLog(request: HttpServletRequest): String = { - val defaultBytes = 100 * 1024 - val appId = Option(request.getParameter("appId")) val executorId = Option(request.getParameter("executorId")) val driverId = Option(request.getParameter("driverId")) @@ -44,9 +43,9 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with val logDir = (appId, executorId, driverId) match { case (Some(a), Some(e), None) => - s"${workDir.getPath}/$appId/$executorId/" + s"${workDir.getPath}/$a/$e/" case (None, None, Some(d)) => - s"${workDir.getPath}/$driverId/" + s"${workDir.getPath}/$d/" case _ => throw new Exception("Request must specify either application or driver identifiers") } @@ -57,7 +56,6 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with } def render(request: HttpServletRequest): Seq[Node] = { - val defaultBytes = 100 * 1024 val appId = Option(request.getParameter("appId")) val executorId = Option(request.getParameter("executorId")) val driverId = Option(request.getParameter("driverId")) @@ -76,49 +74,44 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with val (logText, startByte, endByte, logLength) = getLog(logDir, logType, offset, byteLength) val linkToMaster =

Back to Master

- val range = Bytes {startByte.toString} - {endByte.toString} of {logLength} - - val backButton = - if (startByte > 0) { - - - - } else { - - } + val curLogLength = endByte - startByte + val range = + + Showing {curLogLength} Bytes: {startByte.toString} - {endByte.toString} of {logLength} + + + val moreButton = + + + val newButton = + + + val alert = + - val nextButton = - if (endByte < logLength) { - - - - } else { - - } + val logParams = "?%s&logType=%s".format(params, logType) + val jsOnload = "window.onload = " + + s"initLogPage('$logParams', $curLogLength, $startByte, $endByte, $logLength, $byteLength);" val content =
{linkToMaster} -
-
{backButton}
-
{range}
-
{nextButton}
-
-
-
+ {range} +
+
{moreButton}
{logText}
+ {alert} +
{newButton}
+
+ UIUtils.basicSparkPage(content, logType + " log page for " + pageName) } diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 119165f724f59..db24f0319ba05 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -84,9 +84,7 @@ private[spark] object JettyUtils extends Logging { val result = servletParams.responder(request) response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") response.setHeader("X-Frame-Options", xFrameOptionsValue) - // scalastyle:off println - response.getWriter.println(servletParams.extractFn(result)) - // scalastyle:on println + response.getWriter.print(servletParams.extractFn(result)) } else { response.setStatus(HttpServletResponse.SC_UNAUTHORIZED) response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 28d277df4ae12..6241593bba32f 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -168,6 +168,7 @@ private[spark] object UIUtils extends Logging { + } def vizHeaderNodes: Seq[Node] = { From 80bf48f437939ddc3bb82c8c7530c8ae419f8427 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Wed, 20 Apr 2016 10:32:01 -0700 Subject: [PATCH 55/69] [SPARK-14555] First cut of Python API for Structured Streaming ## What changes were proposed in this pull request? This patch provides a first cut of python APIs for structured streaming. This PR provides the new classes: - ContinuousQuery - Trigger - ProcessingTime in pyspark under `pyspark.sql.streaming`. In addition, it contains the new methods added under: - `DataFrameWriter` a) `startStream` b) `trigger` c) `queryName` - `DataFrameReader` a) `stream` - `DataFrame` a) `isStreaming` This PR doesn't contain all methods exposed for `ContinuousQuery`, for example: - `exception` - `sourceStatuses` - `sinkStatus` They may be added in a follow up. This PR also contains some very minor doc fixes in the Scala side. ## How was this patch tested? Python doc tests TODO: - [ ] verify Python docs look good Author: Burak Yavuz Author: Burak Yavuz Closes #12320 from brkyvz/stream-python. --- python/pyspark/__init__.py | 15 +++ python/pyspark/ml/classification.py | 1 + python/pyspark/ml/clustering.py | 2 +- python/pyspark/ml/evaluation.py | 3 +- python/pyspark/ml/feature.py | 4 +- python/pyspark/ml/pipeline.py | 5 +- python/pyspark/ml/recommendation.py | 2 +- python/pyspark/ml/regression.py | 2 +- python/pyspark/ml/tests.py | 2 +- python/pyspark/ml/tuning.py | 4 +- python/pyspark/ml/util.py | 15 --- python/pyspark/sql/dataframe.py | 12 ++ python/pyspark/sql/readwriter.py | 121 ++++++++++++++++- python/pyspark/sql/streaming.py | 124 ++++++++++++++++++ python/pyspark/sql/tests.py | 93 +++++++++++++ .../test_support/sql/streaming/text-test.txt | 2 + .../apache/spark/sql/ContinuousQuery.scala | 4 +- .../apache/spark/sql/DataFrameWriter.scala | 8 +- .../scala/org/apache/spark/sql/Trigger.scala | 26 ++-- 19 files changed, 397 insertions(+), 48 deletions(-) create mode 100644 python/pyspark/sql/streaming.py create mode 100644 python/test_support/sql/streaming/text-test.txt diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index 111ebaafee3e1..ec1687415a7f6 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -37,6 +37,7 @@ """ +from functools import wraps import types from pyspark.conf import SparkConf @@ -84,6 +85,20 @@ def copy_func(f, name=None, sinceversion=None, doc=None): return fn +def keyword_only(func): + """ + A decorator that forces keyword arguments in the wrapped method + and saves actual input keyword arguments in `_input_kwargs`. + """ + @wraps(func) + def wrapper(*args, **kwargs): + if len(args) > 1: + raise TypeError("Method %s forces keyword arguments." % func.__name__) + wrapper._input_kwargs = kwargs + return func(*args, **kwargs) + return wrapper + + # for back compatibility from pyspark.sql import SQLContext, HiveContext, Row diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index de1321b139751..cc562d2d3d7e6 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -18,6 +18,7 @@ import operator import warnings +from pyspark import since, keyword_only from pyspark.ml import Estimator, Model from pyspark.ml.param.shared import * from pyspark.ml.regression import ( diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index 05aa2dfe74b7e..ecdaa3a71c0a1 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -15,7 +15,7 @@ # limitations under the License. # -from pyspark import since +from pyspark import since, keyword_only from pyspark.ml.util import * from pyspark.ml.wrapper import JavaEstimator, JavaModel from pyspark.ml.param.shared import * diff --git a/python/pyspark/ml/evaluation.py b/python/pyspark/ml/evaluation.py index 52a3fe8985747..455795f9a083a 100644 --- a/python/pyspark/ml/evaluation.py +++ b/python/pyspark/ml/evaluation.py @@ -17,11 +17,10 @@ from abc import abstractmethod, ABCMeta -from pyspark import since +from pyspark import since, keyword_only from pyspark.ml.wrapper import JavaParams from pyspark.ml.param import Param, Params, TypeConverters from pyspark.ml.param.shared import HasLabelCol, HasPredictionCol, HasRawPredictionCol -from pyspark.ml.util import keyword_only from pyspark.mllib.common import inherit_doc __all__ = ['Evaluator', 'BinaryClassificationEvaluator', 'RegressionEvaluator', diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 4310f154b5186..a1911cebe3fbf 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -21,10 +21,10 @@ from py4j.java_collections import JavaArray -from pyspark import since +from pyspark import since, keyword_only from pyspark.rdd import ignore_unicode_prefix from pyspark.ml.param.shared import * -from pyspark.ml.util import keyword_only, JavaMLReadable, JavaMLWritable +from pyspark.ml.util import JavaMLReadable, JavaMLWritable from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaTransformer, _jvm from pyspark.mllib.common import inherit_doc from pyspark.mllib.linalg import _convert_to_vector diff --git a/python/pyspark/ml/pipeline.py b/python/pyspark/ml/pipeline.py index e2651aebdfd71..146e403a8f97b 100644 --- a/python/pyspark/ml/pipeline.py +++ b/python/pyspark/ml/pipeline.py @@ -20,11 +20,10 @@ if sys.version > '3': basestring = str -from pyspark import SparkContext -from pyspark import since +from pyspark import since, keyword_only, SparkContext from pyspark.ml import Estimator, Model, Transformer from pyspark.ml.param import Param, Params -from pyspark.ml.util import keyword_only, JavaMLWriter, JavaMLReader, MLReadable, MLWritable +from pyspark.ml.util import JavaMLWriter, JavaMLReader, MLReadable, MLWritable from pyspark.ml.wrapper import JavaParams from pyspark.mllib.common import inherit_doc diff --git a/python/pyspark/ml/recommendation.py b/python/pyspark/ml/recommendation.py index 9d7f22a66fde6..4e42c468cc42c 100644 --- a/python/pyspark/ml/recommendation.py +++ b/python/pyspark/ml/recommendation.py @@ -15,7 +15,7 @@ # limitations under the License. # -from pyspark import since +from pyspark import since, keyword_only from pyspark.ml.util import * from pyspark.ml.wrapper import JavaEstimator, JavaModel from pyspark.ml.param.shared import * diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 8b68622524304..da74ab5070797 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -17,7 +17,7 @@ import warnings -from pyspark import since +from pyspark import since, keyword_only from pyspark.ml.param.shared import * from pyspark.ml.util import * from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaWrapper diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index f1bca6ebe0ce7..e95458699dd28 100644 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -42,6 +42,7 @@ import tempfile import numpy as np +from pyspark import keyword_only from pyspark.ml import Estimator, Model, Pipeline, PipelineModel, Transformer from pyspark.ml.classification import ( LogisticRegression, DecisionTreeClassifier, OneVsRest, OneVsRestModel) @@ -52,7 +53,6 @@ from pyspark.ml.param.shared import HasMaxIter, HasInputCol, HasSeed from pyspark.ml.regression import LinearRegression, DecisionTreeRegressor from pyspark.ml.tuning import * -from pyspark.ml.util import keyword_only from pyspark.ml.util import MLWritable, MLWriter from pyspark.ml.wrapper import JavaParams from pyspark.mllib.linalg import Vectors, DenseVector, SparseVector diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index 5ac539eddea5d..ef14da488e44f 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -19,11 +19,11 @@ import numpy as np from pyspark import SparkContext -from pyspark import since +from pyspark import since, keyword_only from pyspark.ml import Estimator, Model from pyspark.ml.param import Params, Param, TypeConverters from pyspark.ml.param.shared import HasSeed -from pyspark.ml.util import keyword_only, JavaMLWriter, JavaMLReader, MLReadable, MLWritable +from pyspark.ml.util import JavaMLWriter, JavaMLReader, MLReadable, MLWritable from pyspark.ml.wrapper import JavaParams from pyspark.sql.functions import rand from pyspark.mllib.common import inherit_doc, _py2java diff --git a/python/pyspark/ml/util.py b/python/pyspark/ml/util.py index 841bfb47e1b9d..7003e587ad0eb 100644 --- a/python/pyspark/ml/util.py +++ b/python/pyspark/ml/util.py @@ -17,7 +17,6 @@ import sys import uuid -from functools import wraps if sys.version > '3': basestring = str @@ -39,20 +38,6 @@ def _jvm(): raise AttributeError("Cannot load _jvm from SparkContext. Is SparkContext initialized?") -def keyword_only(func): - """ - A decorator that forces keyword arguments in the wrapped method - and saves actual input keyword arguments in `_input_kwargs`. - """ - @wraps(func) - def wrapper(*args, **kwargs): - if len(args) > 1: - raise TypeError("Method %s forces keyword arguments." % func.__name__) - wrapper._input_kwargs = kwargs - return func(*args, **kwargs) - return wrapper - - class Identifiable(object): """ Object with a unique ID. diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 328bda6601565..bbe15f5f900da 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -197,6 +197,18 @@ def isLocal(self): """ return self._jdf.isLocal() + @property + @since(2.0) + def isStreaming(self): + """Returns true if this :class:`Dataset` contains one or more sources that continuously + return data as it arrives. A :class:`Dataset` that reads data from a streaming source + must be executed as a :class:`ContinuousQuery` using the :func:`startStream` method in + :class:`DataFrameWriter`. Methods that return a single answer, (e.g., :func:`count` or + :func:`collect`) will throw an :class:`AnalysisException` when there is a streaming + source present. + """ + return self._jdf.isStreaming() + @since(1.3) def show(self, n=20, truncate=True): """Prints the first ``n`` rows to the console. diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 0cef37e57cd54..6c809d1139b2d 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -22,7 +22,7 @@ from py4j.java_gateway import JavaClass -from pyspark import RDD, since +from pyspark import RDD, since, keyword_only from pyspark.rdd import ignore_unicode_prefix from pyspark.sql.column import _to_seq from pyspark.sql.types import * @@ -136,6 +136,32 @@ def load(self, path=None, format=None, schema=None, **options): else: return self._df(self._jreader.load()) + @since(2.0) + def stream(self, path=None, format=None, schema=None, **options): + """Loads a data stream from a data source and returns it as a :class`DataFrame`. + + :param path: optional string for file-system backed data sources. + :param format: optional string for format of the data source. Default to 'parquet'. + :param schema: optional :class:`StructType` for the input schema. + :param options: all other string options + + >>> df = sqlContext.read.format('text').stream('python/test_support/sql/streaming') + >>> df.isStreaming + True + """ + if format is not None: + self.format(format) + if schema is not None: + self.schema(schema) + self.options(**options) + if path is not None: + if type(path) != str or len(path.strip()) == 0: + raise ValueError("If the path is provided for stream, it needs to be a " + + "non-empty string. List of paths are not supported.") + return self._df(self._jreader.stream(path)) + else: + return self._df(self._jreader.stream()) + @since(1.4) def json(self, path, schema=None): """ @@ -334,6 +360,10 @@ def __init__(self, df): self._sqlContext = df.sql_ctx self._jwrite = df._jdf.write() + def _cq(self, jcq): + from pyspark.sql.streaming import ContinuousQuery + return ContinuousQuery(jcq, self._sqlContext) + @since(1.4) def mode(self, saveMode): """Specifies the behavior when data or table already exists. @@ -395,6 +425,44 @@ def partitionBy(self, *cols): self._jwrite = self._jwrite.partitionBy(_to_seq(self._sqlContext._sc, cols)) return self + @since(2.0) + def queryName(self, queryName): + """Specifies the name of the :class:`ContinuousQuery` that can be started with + :func:`startStream`. This name must be unique among all the currently active queries + in the associated SQLContext. + + :param queryName: unique name for the query + + >>> writer = sdf.write.queryName('streaming_query') + """ + if not queryName or type(queryName) != str or len(queryName.strip()) == 0: + raise ValueError('The queryName must be a non-empty string. Got: %s' % queryName) + self._jwrite = self._jwrite.queryName(queryName) + return self + + @keyword_only + @since(2.0) + def trigger(self, processingTime=None): + """Set the trigger for the stream query. If this is not set it will run the query as fast + as possible, which is equivalent to setting the trigger to ``processingTime='0 seconds'``. + + :param processingTime: a processing time interval as a string, e.g. '5 seconds', '1 minute'. + + >>> # trigger the query for execution every 5 seconds + >>> writer = sdf.write.trigger(processingTime='5 seconds') + """ + from pyspark.sql.streaming import ProcessingTime + trigger = None + if processingTime is not None: + if type(processingTime) != str or len(processingTime.strip()) == 0: + raise ValueError('The processing time must be a non empty string. Got: %s' % + processingTime) + trigger = ProcessingTime(processingTime) + if trigger is None: + raise ValueError('A trigger was not provided. Supported triggers: processingTime.') + self._jwrite = self._jwrite.trigger(trigger._to_java_trigger(self._sqlContext)) + return self + @since(1.4) def save(self, path=None, format=None, mode=None, partitionBy=None, **options): """Saves the contents of the :class:`DataFrame` to a data source. @@ -426,6 +494,55 @@ def save(self, path=None, format=None, mode=None, partitionBy=None, **options): else: self._jwrite.save(path) + @ignore_unicode_prefix + @since(2.0) + def startStream(self, path=None, format=None, partitionBy=None, queryName=None, **options): + """Streams the contents of the :class:`DataFrame` to a data source. + + The data source is specified by the ``format`` and a set of ``options``. + If ``format`` is not specified, the default data source configured by + ``spark.sql.sources.default`` will be used. + + :param path: the path in a Hadoop supported file system + :param format: the format used to save + + * ``append``: Append contents of this :class:`DataFrame` to existing data. + * ``overwrite``: Overwrite existing data. + * ``ignore``: Silently ignore this operation if data already exists. + * ``error`` (default case): Throw an exception if data already exists. + :param partitionBy: names of partitioning columns + :param queryName: unique name for the query + :param options: All other string options. You may want to provide a `checkpointLocation` + for most streams, however it is not required for a `memory` stream. + + >>> cq = sdf.write.format('memory').queryName('this_query').startStream() + >>> cq.isActive + True + >>> cq.name + u'this_query' + >>> cq.stop() + >>> cq.isActive + False + >>> cq = sdf.write.trigger(processingTime='5 seconds').startStream( + ... queryName='that_query', format='memory') + >>> cq.name + u'that_query' + >>> cq.isActive + True + >>> cq.stop() + """ + self.options(**options) + if partitionBy is not None: + self.partitionBy(partitionBy) + if format is not None: + self.format(format) + if queryName is not None: + self.queryName(queryName) + if path is None: + return self._cq(self._jwrite.startStream()) + else: + return self._cq(self._jwrite.startStream(path)) + @since(1.4) def insertInto(self, tableName, overwrite=False): """Inserts the content of the :class:`DataFrame` to the specified table. @@ -625,6 +742,8 @@ def _test(): globs['sqlContext'] = SQLContext(sc) globs['hiveContext'] = HiveContext(sc) globs['df'] = globs['sqlContext'].read.parquet('python/test_support/sql/parquet_partitioned') + globs['sdf'] =\ + globs['sqlContext'].read.format('text').stream('python/test_support/sql/streaming') (failure_count, test_count) = doctest.testmod( pyspark.sql.readwriter, globs=globs, diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py new file mode 100644 index 0000000000000..549561669fdad --- /dev/null +++ b/python/pyspark/sql/streaming.py @@ -0,0 +1,124 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from abc import ABCMeta, abstractmethod + +from pyspark import since + +__all__ = ["ContinuousQuery"] + + +class ContinuousQuery(object): + """ + A handle to a query that is executing continuously in the background as new data arrives. + All these methods are thread-safe. + + .. note:: Experimental + + .. versionadded:: 2.0 + """ + + def __init__(self, jcq, sqlContext): + self._jcq = jcq + self._sqlContext = sqlContext + + @property + @since(2.0) + def name(self): + """The name of the continuous query. + """ + return self._jcq.name() + + @property + @since(2.0) + def isActive(self): + """Whether this continuous query is currently active or not. + """ + return self._jcq.isActive() + + @since(2.0) + def awaitTermination(self, timeoutMs=None): + """Waits for the termination of `this` query, either by :func:`query.stop()` or by an + exception. If the query has terminated with an exception, then the exception will be thrown. + If `timeoutMs` is set, it returns whether the query has terminated or not within the + `timeoutMs` milliseconds. + + If the query has terminated, then all subsequent calls to this method will either return + immediately (if the query was terminated by :func:`stop()`), or throw the exception + immediately (if the query has terminated with exception). + + throws ContinuousQueryException, if `this` query has terminated with an exception + """ + if timeoutMs is not None: + if type(timeoutMs) != int or timeoutMs < 0: + raise ValueError("timeoutMs must be a positive integer. Got %s" % timeoutMs) + return self._jcq.awaitTermination(timeoutMs) + else: + return self._jcq.awaitTermination() + + @since(2.0) + def processAllAvailable(self): + """Blocks until all available data in the source has been processed an committed to the + sink. This method is intended for testing. Note that in the case of continually arriving + data, this method may block forever. Additionally, this method is only guaranteed to block + until data that has been synchronously appended data to a stream source prior to invocation. + (i.e. `getOffset` must immediately reflect the addition). + """ + return self._jcq.processAllAvailable() + + @since(2.0) + def stop(self): + """Stop this continuous query. + """ + self._jcq.stop() + + +class Trigger(object): + """Used to indicate how often results should be produced by a :class:`ContinuousQuery`. + + .. note:: Experimental + + .. versionadded:: 2.0 + """ + + __metaclass__ = ABCMeta + + @abstractmethod + def _to_java_trigger(self, sqlContext): + """Internal method to construct the trigger on the jvm. + """ + pass + + +class ProcessingTime(Trigger): + """A trigger that runs a query periodically based on the processing time. If `interval` is 0, + the query will run as fast as possible. + + The interval should be given as a string, e.g. '2 seconds', '5 minutes', ... + + .. note:: Experimental + + .. versionadded:: 2.0 + """ + + def __init__(self, interval): + if interval is None or type(interval) != str or len(interval.strip()) == 0: + raise ValueError("interval should be a non empty interval string, e.g. '2 seconds'.") + self.interval = interval + + def _to_java_trigger(self, sqlContext): + return sqlContext._sc._jvm.org.apache.spark.sql.ProcessingTime.create(self.interval) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index d4c221d7125ca..1e864b4cd1bd7 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -879,6 +879,99 @@ def test_save_and_load_builder(self): shutil.rmtree(tmpPath) + def test_stream_trigger_takes_keyword_args(self): + df = self.sqlCtx.read.format('text').stream('python/test_support/sql/streaming') + try: + df.write.trigger('5 seconds') + self.fail("Should have thrown an exception") + except TypeError: + # should throw error + pass + + def test_stream_read_options(self): + schema = StructType([StructField("data", StringType(), False)]) + df = self.sqlCtx.read.format('text').option('path', 'python/test_support/sql/streaming')\ + .schema(schema).stream() + self.assertTrue(df.isStreaming) + self.assertEqual(df.schema.simpleString(), "struct") + + def test_stream_read_options_overwrite(self): + bad_schema = StructType([StructField("test", IntegerType(), False)]) + schema = StructType([StructField("data", StringType(), False)]) + df = self.sqlCtx.read.format('csv').option('path', 'python/test_support/sql/fake') \ + .schema(bad_schema).stream(path='python/test_support/sql/streaming', + schema=schema, format='text') + self.assertTrue(df.isStreaming) + self.assertEqual(df.schema.simpleString(), "struct") + + def test_stream_save_options(self): + df = self.sqlCtx.read.format('text').stream('python/test_support/sql/streaming') + tmpPath = tempfile.mkdtemp() + shutil.rmtree(tmpPath) + self.assertTrue(df.isStreaming) + out = os.path.join(tmpPath, 'out') + chk = os.path.join(tmpPath, 'chk') + cq = df.write.option('checkpointLocation', chk).queryName('this_query')\ + .format('parquet').option('path', out).startStream() + self.assertEqual(cq.name, 'this_query') + self.assertTrue(cq.isActive) + cq.processAllAvailable() + output_files = [] + for _, _, files in os.walk(out): + output_files.extend([f for f in files if 'parquet' in f and not f.startswith('.')]) + self.assertTrue(len(output_files) > 0) + self.assertTrue(len(os.listdir(chk)) > 0) + cq.stop() + shutil.rmtree(tmpPath) + + def test_stream_save_options_overwrite(self): + df = self.sqlCtx.read.format('text').stream('python/test_support/sql/streaming') + tmpPath = tempfile.mkdtemp() + shutil.rmtree(tmpPath) + self.assertTrue(df.isStreaming) + out = os.path.join(tmpPath, 'out') + chk = os.path.join(tmpPath, 'chk') + fake1 = os.path.join(tmpPath, 'fake1') + fake2 = os.path.join(tmpPath, 'fake2') + cq = df.write.option('checkpointLocation', fake1).format('memory').option('path', fake2) \ + .queryName('fake_query').startStream(path=out, format='parquet', queryName='this_query', + checkpointLocation=chk) + self.assertEqual(cq.name, 'this_query') + self.assertTrue(cq.isActive) + cq.processAllAvailable() + output_files = [] + for _, _, files in os.walk(out): + output_files.extend([f for f in files if 'parquet' in f and not f.startswith('.')]) + self.assertTrue(len(output_files) > 0) + self.assertTrue(len(os.listdir(chk)) > 0) + self.assertFalse(os.path.isdir(fake1)) # should not have been created + self.assertFalse(os.path.isdir(fake2)) # should not have been created + cq.stop() + shutil.rmtree(tmpPath) + + def test_stream_await_termination(self): + df = self.sqlCtx.read.format('text').stream('python/test_support/sql/streaming') + tmpPath = tempfile.mkdtemp() + shutil.rmtree(tmpPath) + self.assertTrue(df.isStreaming) + out = os.path.join(tmpPath, 'out') + chk = os.path.join(tmpPath, 'chk') + cq = df.write.startStream(path=out, format='parquet', queryName='this_query', + checkpointLocation=chk) + self.assertTrue(cq.isActive) + try: + cq.awaitTermination("hello") + self.fail("Expected a value exception") + except ValueError: + pass + now = time.time() + res = cq.awaitTermination(2600) # test should take at least 2 seconds + duration = time.time() - now + self.assertTrue(duration >= 2) + self.assertFalse(res) + cq.stop() + shutil.rmtree(tmpPath) + def test_help_command(self): # Regression test for SPARK-5464 rdd = self.sc.parallelize(['{"foo":"bar"}', '{"foo":"baz"}']) diff --git a/python/test_support/sql/streaming/text-test.txt b/python/test_support/sql/streaming/text-test.txt new file mode 100644 index 0000000000000..ae1e76c9e93a7 --- /dev/null +++ b/python/test_support/sql/streaming/text-test.txt @@ -0,0 +1,2 @@ +hello +this \ No newline at end of file diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQuery.scala index d9973b092dc11..953169b63604f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQuery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQuery.scala @@ -56,7 +56,7 @@ trait ContinuousQuery { * Returns current status of all the sources. * @since 2.0.0 */ - def sourceStatuses: Array[SourceStatus] + def sourceStatuses: Array[SourceStatus] /** Returns current status of the sink. */ def sinkStatus: SinkStatus @@ -77,7 +77,7 @@ trait ContinuousQuery { /** * Waits for the termination of `this` query, either by `query.stop()` or by an exception. - * If the query has terminated with an exception, then the exception will be throw. + * If the query has terminated with an exception, then the exception will be thrown. * Otherwise, it returns whether the query has terminated or not within the `timeoutMs` * milliseconds. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 1deeb8a2d25c2..0745ef47ffdac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -85,18 +85,18 @@ final class DataFrameWriter private[sql](df: DataFrame) { * * Scala Example: * {{{ - * def.writer.trigger(ProcessingTime("10 seconds")) + * df.write.trigger(ProcessingTime("10 seconds")) * * import scala.concurrent.duration._ - * def.writer.trigger(ProcessingTime(10.seconds)) + * df.write.trigger(ProcessingTime(10.seconds)) * }}} * * Java Example: * {{{ - * def.writer.trigger(ProcessingTime.create("10 seconds")) + * df.write.trigger(ProcessingTime.create("10 seconds")) * * import java.util.concurrent.TimeUnit - * def.writer.trigger(ProcessingTime.create(10, TimeUnit.SECONDS)) + * df.write.trigger(ProcessingTime.create(10, TimeUnit.SECONDS)) * }}} * * @since 2.0.0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Trigger.scala b/sql/core/src/main/scala/org/apache/spark/sql/Trigger.scala index c4e54b3f90ac5..256e8a47a4665 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Trigger.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Trigger.scala @@ -35,23 +35,23 @@ sealed trait Trigger {} /** * :: Experimental :: - * A trigger that runs a query periodically based on the processing time. If `intervalMs` is 0, + * A trigger that runs a query periodically based on the processing time. If `interval` is 0, * the query will run as fast as possible. * * Scala Example: * {{{ - * def.writer.trigger(ProcessingTime("10 seconds")) + * df.write.trigger(ProcessingTime("10 seconds")) * * import scala.concurrent.duration._ - * def.writer.trigger(ProcessingTime(10.seconds)) + * df.write.trigger(ProcessingTime(10.seconds)) * }}} * * Java Example: * {{{ - * def.writer.trigger(ProcessingTime.create("10 seconds")) + * df.write.trigger(ProcessingTime.create("10 seconds")) * * import java.util.concurrent.TimeUnit - * def.writer.trigger(ProcessingTime.create(10, TimeUnit.SECONDS)) + * df.write.trigger(ProcessingTime.create(10, TimeUnit.SECONDS)) * }}} */ @Experimental @@ -67,11 +67,11 @@ case class ProcessingTime(intervalMs: Long) extends Trigger { object ProcessingTime { /** - * Create a [[ProcessingTime]]. If `intervalMs` is 0, the query will run as fast as possible. + * Create a [[ProcessingTime]]. If `interval` is 0, the query will run as fast as possible. * * Example: * {{{ - * def.writer.trigger(ProcessingTime("10 seconds")) + * df.write.trigger(ProcessingTime("10 seconds")) * }}} */ def apply(interval: String): ProcessingTime = { @@ -94,12 +94,12 @@ object ProcessingTime { } /** - * Create a [[ProcessingTime]]. If `intervalMs` is 0, the query will run as fast as possible. + * Create a [[ProcessingTime]]. If `interval` is 0, the query will run as fast as possible. * * Example: * {{{ * import scala.concurrent.duration._ - * def.writer.trigger(ProcessingTime(10.seconds)) + * df.write.trigger(ProcessingTime(10.seconds)) * }}} */ def apply(interval: Duration): ProcessingTime = { @@ -107,11 +107,11 @@ object ProcessingTime { } /** - * Create a [[ProcessingTime]]. If `intervalMs` is 0, the query will run as fast as possible. + * Create a [[ProcessingTime]]. If `interval` is 0, the query will run as fast as possible. * * Example: * {{{ - * def.writer.trigger(ProcessingTime.create("10 seconds")) + * df.write.trigger(ProcessingTime.create("10 seconds")) * }}} */ def create(interval: String): ProcessingTime = { @@ -119,12 +119,12 @@ object ProcessingTime { } /** - * Create a [[ProcessingTime]]. If `intervalMs` is 0, the query will run as fast as possible. + * Create a [[ProcessingTime]]. If `interval` is 0, the query will run as fast as possible. * * Example: * {{{ * import java.util.concurrent.TimeUnit - * def.writer.trigger(ProcessingTime.create(10, TimeUnit.SECONDS)) + * df.write.trigger(ProcessingTime.create(10, TimeUnit.SECONDS)) * }}} */ def create(interval: Long, unit: TimeUnit): ProcessingTime = { From b4e76a9a3b58822fcbe5a8b137618a32c4033755 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 20 Apr 2016 10:46:02 -0700 Subject: [PATCH 56/69] [SPARK-14742][DOCS] Redirect spark-ec2 doc to new location ## What changes were proposed in this pull request? Restore `ec2-scripts.md` as a redirect to amplab/spark-ec2 docs ## How was this patch tested? `jekyll build` and checked with the browser Author: Sean Owen Closes #12534 from srowen/SPARK-14742. --- docs/ec2-scripts.md | 7 +++++++ 1 file changed, 7 insertions(+) create mode 100644 docs/ec2-scripts.md diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md new file mode 100644 index 0000000000000..6cd39dbed055d --- /dev/null +++ b/docs/ec2-scripts.md @@ -0,0 +1,7 @@ +--- +layout: global +title: Running Spark on EC2 +redirect: https://github.com/amplab/spark-ec2#readme +--- + +This document has been superseded and replaced by documentation at https://github.com/amplab/spark-ec2#readme From 90cbc82fd4114219a5a0f180b1908a18985fda3e Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 20 Apr 2016 10:48:11 -0700 Subject: [PATCH 57/69] [SPARK-14725][CORE] Remove HttpServer class ## What changes were proposed in this pull request? This proposal removes the class `HttpServer`, with the changing of internal file/jar/class transmission to RPC layer, currently there's no code using this `HttpServer`, so here propose to remove it. ## How was this patch tested? Unit test is verified locally. Author: jerryshao Closes #12526 from jerryshao/SPARK-14725. --- .../scala/org/apache/spark/HttpServer.scala | 181 ------------------ .../spark/repl/ExecutorClassLoaderSuite.scala | 53 ----- 2 files changed, 234 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/HttpServer.scala diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala deleted file mode 100644 index 982b6d6b61732..0000000000000 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ /dev/null @@ -1,181 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark - -import java.io.File - -import org.eclipse.jetty.security.{ConstraintMapping, ConstraintSecurityHandler, HashLoginService} -import org.eclipse.jetty.security.authentication.DigestAuthenticator -import org.eclipse.jetty.server.Server -import org.eclipse.jetty.server.bio.SocketConnector -import org.eclipse.jetty.server.ssl.SslSocketConnector -import org.eclipse.jetty.servlet.{DefaultServlet, ServletContextHandler, ServletHolder} -import org.eclipse.jetty.util.component.LifeCycle -import org.eclipse.jetty.util.security.{Constraint, Password} -import org.eclipse.jetty.util.thread.QueuedThreadPool - -import org.apache.spark.internal.Logging -import org.apache.spark.util.Utils - -/** - * Exception type thrown by HttpServer when it is in the wrong state for an operation. - */ -private[spark] class ServerStateException(message: String) extends Exception(message) - -/** - * An HTTP server for static content used to allow worker nodes to access JARs added to SparkContext - * as well as classes created by the interpreter when the user types in code. This is just a wrapper - * around a Jetty server. - */ -private[spark] class HttpServer( - conf: SparkConf, - resourceBase: File, - securityManager: SecurityManager, - requestedPort: Int = 0, - serverName: String = "HTTP server") - extends Logging { - - private var server: Server = null - private var port: Int = requestedPort - private val servlets = { - val handler = new ServletContextHandler() - handler.setContextPath("/") - handler - } - - def start() { - if (server != null) { - throw new ServerStateException("Server is already started") - } else { - logInfo("Starting HTTP Server") - val (actualServer, actualPort) = - Utils.startServiceOnPort[Server](requestedPort, doStart, conf, serverName) - server = actualServer - port = actualPort - } - } - - def addDirectory(contextPath: String, resourceBase: String): Unit = { - val holder = new ServletHolder() - holder.setInitParameter("resourceBase", resourceBase) - holder.setInitParameter("pathInfoOnly", "true") - holder.setServlet(new DefaultServlet()) - servlets.addServlet(holder, contextPath.stripSuffix("/") + "/*") - } - - /** - * Actually start the HTTP server on the given port. - * - * Note that this is only best effort in the sense that we may end up binding to a nearby port - * in the event of port collision. Return the bound server and the actual port used. - */ - private def doStart(startPort: Int): (Server, Int) = { - val server = new Server() - - val connector = securityManager.fileServerSSLOptions.createJettySslContextFactory() - .map(new SslSocketConnector(_)).getOrElse(new SocketConnector) - - connector.setMaxIdleTime(60 * 1000) - connector.setSoLingerTime(-1) - connector.setPort(startPort) - server.addConnector(connector) - - val threadPool = new QueuedThreadPool - threadPool.setDaemon(true) - server.setThreadPool(threadPool) - addDirectory("/", resourceBase.getAbsolutePath) - - if (securityManager.isAuthenticationEnabled()) { - logDebug("HttpServer is using security") - val sh = setupSecurityHandler(securityManager) - // make sure we go through security handler to get resources - sh.setHandler(servlets) - server.setHandler(sh) - } else { - logDebug("HttpServer is not using security") - server.setHandler(servlets) - } - - server.start() - val actualPort = server.getConnectors()(0).getLocalPort - - (server, actualPort) - } - - /** - * Setup Jetty to the HashLoginService using a single user with our - * shared secret. Configure it to use DIGEST-MD5 authentication so that the password - * isn't passed in plaintext. - */ - private def setupSecurityHandler(securityMgr: SecurityManager): ConstraintSecurityHandler = { - val constraint = new Constraint() - // use DIGEST-MD5 as the authentication mechanism - constraint.setName(Constraint.__DIGEST_AUTH) - constraint.setRoles(Array("user")) - constraint.setAuthenticate(true) - constraint.setDataConstraint(Constraint.DC_NONE) - - val cm = new ConstraintMapping() - cm.setConstraint(constraint) - cm.setPathSpec("/*") - val sh = new ConstraintSecurityHandler() - - // the hashLoginService lets us do a single user and - // secret right now. This could be changed to use the - // JAASLoginService for other options. - val hashLogin = new HashLoginService() - - val userCred = new Password(securityMgr.getSecretKey()) - if (userCred == null) { - throw new Exception("Error: secret key is null with authentication on") - } - hashLogin.putUser(securityMgr.getHttpUser(), userCred, Array("user")) - sh.setLoginService(hashLogin) - sh.setAuthenticator(new DigestAuthenticator()); - sh.setConstraintMappings(Array(cm)) - sh - } - - def stop() { - if (server == null) { - throw new ServerStateException("Server is already stopped") - } else { - server.stop() - // Stop the ThreadPool if it supports stop() method (through LifeCycle). - // It is needed because stopping the Server won't stop the ThreadPool it uses. - val threadPool = server.getThreadPool - if (threadPool != null && threadPool.isInstanceOf[LifeCycle]) { - threadPool.asInstanceOf[LifeCycle].stop - } - port = -1 - server = null - } - } - - /** - * Get the URI of this HTTP server (http://host:port or https://host:port) - */ - def uri: String = { - if (server == null) { - throw new ServerStateException("Server is not started") - } else { - val scheme = if (securityManager.fileServerSSLOptions.enabled) "https" else "http" - s"$scheme://${Utils.localHostNameForURI()}:$port" - } - } -} diff --git a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala index 9a143ee36ff46..12e98565dcef3 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala @@ -57,7 +57,6 @@ class ExecutorClassLoaderSuite var tempDir2: File = _ var url1: String = _ var urls2: Array[URL] = _ - var classServer: HttpServer = _ override def beforeAll() { super.beforeAll() @@ -74,9 +73,6 @@ class ExecutorClassLoaderSuite override def afterAll() { try { - if (classServer != null) { - classServer.stop() - } Utils.deleteRecursively(tempDir1) Utils.deleteRecursively(tempDir2) SparkEnv.set(null) @@ -137,55 +133,6 @@ class ExecutorClassLoaderSuite assert(fileReader.readLine().contains("resource"), "File doesn't contain 'resource'") } - test("failing to fetch classes from HTTP server should not leak resources (SPARK-6209)") { - // This is a regression test for SPARK-6209, a bug where each failed attempt to load a class - // from the driver's class server would leak a HTTP connection, causing the class server's - // thread / connection pool to be exhausted. - val conf = new SparkConf() - val securityManager = new SecurityManager(conf) - classServer = new HttpServer(conf, tempDir1, securityManager) - classServer.start() - // ExecutorClassLoader uses SparkEnv's SecurityManager, so we need to mock this - val mockEnv = mock[SparkEnv] - when(mockEnv.securityManager).thenReturn(securityManager) - SparkEnv.set(mockEnv) - // Create an ExecutorClassLoader that's configured to load classes from the HTTP server - val parentLoader = new URLClassLoader(Array.empty, null) - val classLoader = new ExecutorClassLoader(conf, null, classServer.uri, parentLoader, false) - classLoader.httpUrlConnectionTimeoutMillis = 500 - // Check that this class loader can actually load classes that exist - val fakeClass = classLoader.loadClass("ReplFakeClass2").newInstance() - val fakeClassVersion = fakeClass.toString - assert(fakeClassVersion === "1") - // Try to perform a full GC now, since GC during the test might mask resource leaks - System.gc() - // When the original bug occurs, the test thread becomes blocked in a classloading call - // and does not respond to interrupts. Therefore, use a custom ScalaTest interruptor to - // shut down the HTTP server when the test times out - val interruptor: Interruptor = new Interruptor { - override def apply(thread: Thread): Unit = { - classServer.stop() - classServer = null - thread.interrupt() - } - } - def tryAndFailToLoadABunchOfClasses(): Unit = { - // The number of trials here should be much larger than Jetty's thread / connection limit - // in order to expose thread or connection leaks - for (i <- 1 to 1000) { - if (Thread.currentThread().isInterrupted) { - throw new InterruptedException() - } - // Incorporate the iteration number into the class name in order to avoid any response - // caching that might be added in the future - intercept[ClassNotFoundException] { - classLoader.loadClass(s"ReplFakeClassDoesNotExist$i").newInstance() - } - } - } - failAfter(10 seconds)(tryAndFailToLoadABunchOfClasses())(interruptor) - } - test("fetch classes using Spark's RpcEnv") { val env = mock[SparkEnv] val rpcEnv = mock[RpcEnv] From 08f84d7a9a7429b3d6651b5af4d7740027b53d39 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Wed, 20 Apr 2016 20:06:27 +0200 Subject: [PATCH 58/69] [MINOR][ML][PYSPARK] Fix omissive param setters which should use _set method ## What changes were proposed in this pull request? #11939 make Python param setters use the `_set` method. This PR fix omissive ones. ## How was this patch tested? Existing tests. cc jkbradley sethah Author: Yanbo Liang Closes #12531 from yanboliang/setters-omissive. --- python/pyspark/ml/feature.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index a1911cebe3fbf..1b298e639d3b6 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -340,7 +340,7 @@ def setBinary(self, value): """ Sets the value of :py:attr:`binary`. """ - self._paramMap[self.binary] = value + self._set(binary=value) return self @since("2.0.0") @@ -569,7 +569,7 @@ def setBinary(self, value): """ Sets the value of :py:attr:`binary`. """ - self._paramMap[self.binary] = value + self._set(binary=value) return self @since("2.0.0") From acc7e592c4ee5b4a6f42945329fc289fd11e1793 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 20 Apr 2016 11:48:30 -0700 Subject: [PATCH 59/69] [SPARK-14478][ML][MLLIB][DOC] Doc that StandardScaler uses the corrected sample std ## What changes were proposed in this pull request? Currently, MLlib's StandardScaler scales columns using the corrected standard deviation (sqrt of unbiased variance). This matches what R's scale package does. This PR documents this fact. ## How was this patch tested? doc only Author: Joseph K. Bradley Closes #12519 from jkbradley/scaler-variance-doc. --- .../scala/org/apache/spark/ml/feature/StandardScaler.scala | 5 +++++ .../org/apache/spark/mllib/feature/StandardScaler.scala | 5 +++++ 2 files changed, 10 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala index 118a6e3e6ad44..626e97efb47c6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala @@ -66,6 +66,11 @@ private[feature] trait StandardScalerParams extends Params with HasInputCol with * :: Experimental :: * Standardizes features by removing the mean and scaling to unit variance using column summary * statistics on the samples in the training set. + * + * The "unit std" is computed using the + * [[https://en.wikipedia.org/wiki/Standard_deviation#Corrected_sample_standard_deviation + * corrected sample standard deviation]], + * which is computed as the square root of the unbiased sample variance. */ @Experimental class StandardScaler(override val uid: String) extends Estimator[StandardScalerModel] diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala index 5c35e1b91c9bf..ee97045f34dc8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala @@ -27,6 +27,11 @@ import org.apache.spark.rdd.RDD * Standardizes features by removing the mean and scaling to unit std using column summary * statistics on the samples in the training set. * + * The "unit std" is computed using the + * [[https://en.wikipedia.org/wiki/Standard_deviation#Corrected_sample_standard_deviation + * corrected sample standard deviation]], + * which is computed as the square root of the unbiased sample variance. + * * @param withMean False by default. Centers the data with mean before scaling. It will build a * dense output, so this does not work on sparse input and will raise an exception. * @param withStd True by default. Scales the data to unit standard deviation. From cb8ea9e1f34b9af287b3d10e47f24de4307c63ba Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 20 Apr 2016 12:22:51 -0700 Subject: [PATCH 60/69] [SPARK-14741][SQL] Fixed error in reading json file stream inside a partitioned directory ## What changes were proposed in this pull request? Consider the following directory structure dir/col=X/some-files If we create a text format streaming dataframe on `dir/col=X/` then it should not consider as partitioning in columns. Even though the streaming dataframe does not do so, the generated batch dataframes pick up col as a partitioning columns, causing mismatch streaming source schema and generated df schema. This leads to runtime failure: ``` 18:55:11.262 ERROR org.apache.spark.sql.execution.streaming.StreamExecution: Query query-0 terminated with error java.lang.AssertionError: assertion failed: Invalid batch: c#2 != c#7,type#8 ``` The reason is that the partition inferring code has no idea of a base path, above which it should not search of partitions. This PR makes sure that the batch DF is generated with the basePath set as the original path on which the file stream source is defined. ## How was this patch tested? New unit test Author: Tathagata Das Closes #12517 from tdas/SPARK-14741. --- .../execution/datasources/DataSource.scala | 3 ++- .../sql/streaming/FileStreamSourceSuite.scala | 24 +++++++++++++++++++ 2 files changed, 26 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 23a7071086087..0dfe7dba1e5c8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -186,7 +186,8 @@ case class DataSource( userSpecifiedSchema = Some(dataSchema), className = className, options = - new CaseInsensitiveMap(options.filterKeys(_ != "path"))).resolveRelation())) + new CaseInsensitiveMap( + options.filterKeys(_ != "path") + ("basePath" -> path))).resolveRelation())) } new FileStreamSource( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 73d1b1b1d507d..64cddf0deecb0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -281,6 +281,30 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { Utils.deleteRecursively(tmp) } + + test("reading from json files inside partitioned directory") { + val src = { + val base = Utils.createTempDir(namePrefix = "streaming.src") + new File(base, "type=X") + } + val tmp = Utils.createTempDir(namePrefix = "streaming.tmp") + src.mkdirs() + + + // Add a file so that we can infer its schema + stringToFile(new File(src, "existing"), "{'c': 'drop1'}\n{'c': 'keep2'}\n{'c': 'keep3'}") + + val textSource = createFileStreamSource("json", src.getCanonicalPath) + + // FileStreamSource should infer the column "c" + val filtered = textSource.toDF().filter($"c" contains "keep") + + testStream(filtered)( + AddTextFileData(textSource, "{'c': 'drop4'}\n{'c': 'keep5'}\n{'c': 'keep6'}", src, tmp), + CheckAnswer("keep2", "keep3", "keep5", "keep6") + ) + } + test("read from parquet files") { val src = Utils.createTempDir(namePrefix = "streaming.src") val tmp = Utils.createTempDir(namePrefix = "streaming.tmp") From 8fc267ab3322e46db81e725a5cb1adb5a71b2b4d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 20 Apr 2016 12:58:48 -0700 Subject: [PATCH 61/69] [SPARK-14720][SPARK-13643] Move Hive-specific methods into HiveSessionState and Create a SparkSession class ## What changes were proposed in this pull request? This PR has two main changes. 1. Move Hive-specific methods from HiveContext to HiveSessionState, which help the work of removing HiveContext. 2. Create a SparkSession Class, which will later be the entry point of Spark SQL users. ## How was this patch tested? Existing tests This PR is trying to fix test failures of https://github.com/apache/spark/pull/12485. Author: Andrew Or Author: Yin Huai Closes #12522 from yhuai/spark-session. --- .../spark/internal/config/package.scala | 7 + .../org/apache/spark/sql/SQLContext.scala | 52 ++-- .../org/apache/spark/sql/SparkSession.scala | 100 +++++++ .../spark/sql/internal/SessionState.scala | 54 +++- .../SparkExecuteStatementOperation.scala | 2 +- .../hive/thriftserver/SparkSQLDriver.scala | 7 +- .../sql/hive/thriftserver/SparkSQLEnv.scala | 6 +- .../thriftserver/SparkSQLSessionManager.scala | 2 +- .../server/SparkSQLOperationManager.scala | 2 +- .../execution/HiveCompatibilitySuite.scala | 6 +- .../HiveWindowFunctionQuerySuite.scala | 31 +- .../apache/spark/sql/hive/HiveContext.scala | 221 +------------- .../spark/sql/hive/HiveMetastoreCatalog.scala | 30 +- .../spark/sql/hive/HiveQueryExecution.scala | 66 +++++ .../spark/sql/hive/HiveSessionCatalog.scala | 11 +- .../spark/sql/hive/HiveSessionState.scala | 180 ++++++++++-- .../spark/sql/hive/HiveStrategies.scala | 11 +- .../apache/spark/sql/hive/TableReader.scala | 17 +- .../sql/hive/client/HiveClientImpl.scala | 2 + .../hive/execution/CreateTableAsSelect.scala | 9 +- .../hive/execution/CreateViewAsSelect.scala | 11 +- .../hive/execution/HiveNativeCommand.scala | 8 +- .../sql/hive/execution/HiveSqlParser.scala | 19 +- .../sql/hive/execution/HiveTableScan.scala | 6 +- .../hive/execution/InsertIntoHiveTable.scala | 23 +- .../hive/execution/ScriptTransformation.scala | 9 +- .../spark/sql/hive/execution/commands.scala | 38 ++- .../apache/spark/sql/hive/test/TestHive.scala | 275 +++++++++++------- .../spark/sql/hive/ErrorPositionSuite.scala | 3 +- .../spark/sql/hive/HiveContextSuite.scala | 7 +- .../sql/hive/HiveMetastoreCatalogSuite.scala | 6 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 33 +-- .../spark/sql/hive/MultiDatabaseSuite.scala | 9 +- .../spark/sql/hive/StatisticsSuite.scala | 4 +- .../execution/BigDataBenchmarkSuite.scala | 4 +- .../hive/execution/HiveComparisonTest.scala | 25 +- .../sql/hive/execution/HiveQuerySuite.scala | 6 +- .../sql/hive/execution/HiveSerDeSuite.scala | 4 +- .../sql/hive/execution/PruningSuite.scala | 18 +- .../sql/hive/execution/SQLQuerySuite.scala | 4 +- .../execution/ScriptTransformationSuite.scala | 8 +- .../apache/spark/sql/hive/parquetSuites.scala | 6 +- .../spark/sql/sources/BucketedReadSuite.scala | 2 +- 43 files changed, 797 insertions(+), 547 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQueryExecution.scala diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 94b50ee06520c..2c1e0b71e3613 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -89,4 +89,11 @@ package object config { .stringConf .toSequence .createWithDefault(Nil) + + // Note: This is a SQL config but needs to be in core because the REPL depends on it + private[spark] val CATALOG_IMPLEMENTATION = ConfigBuilder("spark.sql.catalogImplementation") + .internal() + .stringConf + .checkValues(Set("hive", "in-memory")) + .createWithDefault("in-memory") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 781d6998190b2..f3f84144ad93e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -63,14 +63,18 @@ import org.apache.spark.util.Utils * @since 1.0.0 */ class SQLContext private[sql]( - @transient protected[sql] val sharedState: SharedState, + @transient private val sparkSession: SparkSession, val isRootContext: Boolean) extends Logging with Serializable { self => + private[sql] def this(sparkSession: SparkSession) = { + this(sparkSession, true) + } + def this(sc: SparkContext) = { - this(new SharedState(sc), true) + this(new SparkSession(sc)) } def this(sparkContext: JavaSparkContext) = this(sparkContext.sc) @@ -97,12 +101,15 @@ class SQLContext private[sql]( } } - def sparkContext: SparkContext = sharedState.sparkContext - + protected[sql] def sessionState: SessionState = sparkSession.sessionState + protected[sql] def sharedState: SharedState = sparkSession.sharedState + protected[sql] def conf: SQLConf = sessionState.conf protected[sql] def cacheManager: CacheManager = sharedState.cacheManager protected[sql] def listener: SQLListener = sharedState.listener protected[sql] def externalCatalog: ExternalCatalog = sharedState.externalCatalog + def sparkContext: SparkContext = sharedState.sparkContext + /** * Returns a [[SQLContext]] as new session, with separated SQL configurations, temporary * tables, registered functions, but sharing the same [[SparkContext]], cached data and @@ -110,14 +117,9 @@ class SQLContext private[sql]( * * @since 1.6.0 */ - def newSession(): SQLContext = new SQLContext(sharedState, isRootContext = false) - - /** - * Per-session state, e.g. configuration, functions, temporary tables etc. - */ - @transient - protected[sql] lazy val sessionState: SessionState = new SessionState(self) - protected[spark] def conf: SQLConf = sessionState.conf + def newSession(): SQLContext = { + new SQLContext(sparkSession.newSession(), isRootContext = false) + } /** * An interface to register custom [[org.apache.spark.sql.util.QueryExecutionListener]]s @@ -132,10 +134,14 @@ class SQLContext private[sql]( * @group config * @since 1.0.0 */ - def setConf(props: Properties): Unit = conf.setConf(props) + def setConf(props: Properties): Unit = sessionState.setConf(props) - /** Set the given Spark SQL configuration property. */ - private[sql] def setConf[T](entry: ConfigEntry[T], value: T): Unit = conf.setConf(entry, value) + /** + * Set the given Spark SQL configuration property. + */ + private[sql] def setConf[T](entry: ConfigEntry[T], value: T): Unit = { + sessionState.setConf(entry, value) + } /** * Set the given Spark SQL configuration property. @@ -143,7 +149,7 @@ class SQLContext private[sql]( * @group config * @since 1.0.0 */ - def setConf(key: String, value: String): Unit = conf.setConfString(key, value) + def setConf(key: String, value: String): Unit = sessionState.setConf(key, value) /** * Return the value of Spark SQL configuration property for the given key. @@ -186,23 +192,19 @@ class SQLContext private[sql]( */ def getAllConfs: immutable.Map[String, String] = conf.getAllConfs - // Extract `spark.sql.*` entries and put it in our SQLConf. - // Subclasses may additionally set these entries in other confs. - SQLContext.getSQLProperties(sparkContext.getConf).asScala.foreach { case (k, v) => - setConf(k, v) - } - protected[sql] def parseSql(sql: String): LogicalPlan = sessionState.sqlParser.parsePlan(sql) protected[sql] def executeSql(sql: String): QueryExecution = executePlan(parseSql(sql)) - protected[sql] def executePlan(plan: LogicalPlan) = new QueryExecution(this, plan) + protected[sql] def executePlan(plan: LogicalPlan): QueryExecution = { + sessionState.executePlan(plan) + } /** * Add a jar to SQLContext */ protected[sql] def addJar(path: String): Unit = { - sparkContext.addJar(path) + sessionState.addJar(path) } /** A [[FunctionResourceLoader]] that can be used in SessionCatalog. */ @@ -768,7 +770,7 @@ class SQLContext private[sql]( * as Spark can parse all supported Hive DDLs itself. */ private[sql] def runNativeSql(sqlText: String): Seq[Row] = { - throw new UnsupportedOperationException + sessionState.runNativeSql(sqlText).map { r => Row(r) } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala new file mode 100644 index 0000000000000..17ba2998250f6 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import scala.reflect.ClassTag +import scala.util.control.NonFatal + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.internal.config.CATALOG_IMPLEMENTATION +import org.apache.spark.sql.internal.{SessionState, SharedState} +import org.apache.spark.util.Utils + + +/** + * The entry point to Spark execution. + */ +class SparkSession private( + sparkContext: SparkContext, + existingSharedState: Option[SharedState]) { self => + + def this(sc: SparkContext) { + this(sc, None) + } + + /** + * Start a new session where configurations, temp tables, temp functions etc. are isolated. + */ + def newSession(): SparkSession = { + // Note: materialize the shared state here to ensure the parent and child sessions are + // initialized with the same shared state. + new SparkSession(sparkContext, Some(sharedState)) + } + + @transient + protected[sql] lazy val sharedState: SharedState = { + existingSharedState.getOrElse( + SparkSession.reflect[SharedState, SparkContext]( + SparkSession.sharedStateClassName(sparkContext.conf), + sparkContext)) + } + + @transient + protected[sql] lazy val sessionState: SessionState = { + SparkSession.reflect[SessionState, SQLContext]( + SparkSession.sessionStateClassName(sparkContext.conf), + new SQLContext(self, isRootContext = false)) + } + +} + + +private object SparkSession { + + private def sharedStateClassName(conf: SparkConf): String = { + conf.get(CATALOG_IMPLEMENTATION) match { + case "hive" => "org.apache.spark.sql.hive.HiveSharedState" + case "in-memory" => classOf[SharedState].getCanonicalName + } + } + + private def sessionStateClassName(conf: SparkConf): String = { + conf.get(CATALOG_IMPLEMENTATION) match { + case "hive" => "org.apache.spark.sql.hive.HiveSessionState" + case "in-memory" => classOf[SessionState].getCanonicalName + } + } + + /** + * Helper method to create an instance of [[T]] using a single-arg constructor that + * accepts an [[Arg]]. + */ + private def reflect[T, Arg <: AnyRef]( + className: String, + ctorArg: Arg)(implicit ctorArgTag: ClassTag[Arg]): T = { + try { + val clazz = Utils.classForName(className) + val ctor = clazz.getDeclaredConstructor(ctorArgTag.runtimeClass) + ctor.newInstance(ctorArg).asInstanceOf[T] + } catch { + case NonFatal(e) => + throw new IllegalArgumentException(s"Error while instantiating '$className':", e) + } + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index d404a7c0aef59..42915d5887f44 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -17,15 +17,22 @@ package org.apache.spark.sql.internal +import java.util.Properties + +import scala.collection.JavaConverters._ + +import org.apache.spark.internal.config.ConfigEntry import org.apache.spark.sql.{ContinuousQueryManager, ExperimentalMethods, SQLContext, UDFRegistration} import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry} import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources.{DataSourceAnalysis, PreInsertCastAndRename, ResolveDataSource} import org.apache.spark.sql.util.ExecutionListenerManager + /** * A class that holds all session-specific state in a given [[SQLContext]]. */ @@ -37,7 +44,10 @@ private[sql] class SessionState(ctx: SQLContext) { /** * SQL-specific key-value configurations. */ - lazy val conf = new SQLConf + lazy val conf: SQLConf = new SQLConf + + // Automatically extract `spark.sql.*` entries and put it in our SQLConf + setConf(SQLContext.getSQLProperties(ctx.sparkContext.getConf)) lazy val experimentalMethods = new ExperimentalMethods @@ -101,5 +111,45 @@ private[sql] class SessionState(ctx: SQLContext) { * Interface to start and stop [[org.apache.spark.sql.ContinuousQuery]]s. */ lazy val continuousQueryManager: ContinuousQueryManager = new ContinuousQueryManager(ctx) -} + + // ------------------------------------------------------ + // Helper methods, partially leftover from pre-2.0 days + // ------------------------------------------------------ + + def executePlan(plan: LogicalPlan): QueryExecution = new QueryExecution(ctx, plan) + + def refreshTable(tableName: String): Unit = { + catalog.refreshTable(sqlParser.parseTableIdentifier(tableName)) + } + + def invalidateTable(tableName: String): Unit = { + catalog.invalidateTable(sqlParser.parseTableIdentifier(tableName)) + } + + final def setConf(properties: Properties): Unit = { + properties.asScala.foreach { case (k, v) => setConf(k, v) } + } + + final def setConf[T](entry: ConfigEntry[T], value: T): Unit = { + conf.setConf(entry, value) + setConf(entry.key, entry.stringConverter(value)) + } + + def setConf(key: String, value: String): Unit = { + conf.setConfString(key, value) + } + + def addJar(path: String): Unit = { + ctx.sparkContext.addJar(path) + } + + def analyze(tableName: String): Unit = { + throw new UnsupportedOperationException + } + + def runNativeSql(sql: String): Seq[String] = { + throw new UnsupportedOperationException + } + +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 673a293ce2601..d89c3b4ab2d1c 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -195,7 +195,7 @@ private[hive] class SparkExecuteStatementOperation( setState(OperationState.RUNNING) // Always use the latest class loader provided by executionHive's state. val executionHiveClassLoader = - hiveContext.executionHive.state.getConf.getClassLoader + hiveContext.sessionState.executionHive.state.getConf.getClassLoader Thread.currentThread().setContextClassLoader(executionHiveClassLoader) HiveThriftServer2.listener.onStatementStart( diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala index b8bc8ea44dc84..7e8eada5adb4f 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} +import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes, HiveQueryExecution} private[hive] class SparkSQLDriver( val context: HiveContext = SparkSQLEnv.hiveContext) @@ -41,7 +41,7 @@ private[hive] class SparkSQLDriver( override def init(): Unit = { } - private def getResultSetSchema(query: context.QueryExecution): Schema = { + private def getResultSetSchema(query: HiveQueryExecution): Schema = { val analyzed = query.analyzed logDebug(s"Result Schema: ${analyzed.output}") if (analyzed.output.isEmpty) { @@ -59,7 +59,8 @@ private[hive] class SparkSQLDriver( // TODO unify the error code try { context.sparkContext.setJobDescription(command) - val execution = context.executePlan(context.sql(command).logicalPlan) + val execution = + context.executePlan(context.sql(command).logicalPlan).asInstanceOf[HiveQueryExecution] hiveResponse = execution.stringResult() tableSchema = getResultSetSchema(execution) new CommandProcessorResponse(0) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index ae1d737b58adc..2679ac1854bb8 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -58,9 +58,9 @@ private[hive] object SparkSQLEnv extends Logging { sparkContext.addSparkListener(new StatsReportListener()) hiveContext = new HiveContext(sparkContext) - hiveContext.metadataHive.setOut(new PrintStream(System.out, true, "UTF-8")) - hiveContext.metadataHive.setInfo(new PrintStream(System.err, true, "UTF-8")) - hiveContext.metadataHive.setError(new PrintStream(System.err, true, "UTF-8")) + hiveContext.sessionState.metadataHive.setOut(new PrintStream(System.out, true, "UTF-8")) + hiveContext.sessionState.metadataHive.setInfo(new PrintStream(System.err, true, "UTF-8")) + hiveContext.sessionState.metadataHive.setError(new PrintStream(System.err, true, "UTF-8")) hiveContext.setConf("spark.sql.hive.version", HiveContext.hiveExecutionVersion) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala index de4e9c62b57a4..f492b5656c3c3 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala @@ -71,7 +71,7 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, hiveContext: val session = super.getSession(sessionHandle) HiveThriftServer2.listener.onSessionCreated( session.getIpAddress, sessionHandle.getSessionId.toString, session.getUsername) - val ctx = if (hiveContext.hiveThriftServerSingleSession) { + val ctx = if (hiveContext.sessionState.hiveThriftServerSingleSession) { hiveContext } else { hiveContext.newSession() diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index 0c468a408ba98..da410c68c851d 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -47,7 +47,7 @@ private[thriftserver] class SparkSQLOperationManager() confOverlay: JMap[String, String], async: Boolean): ExecuteStatementOperation = synchronized { val hiveContext = sessionToContexts(parentSession.getSessionHandle) - val runInBackground = async && hiveContext.hiveThriftServerAsync + val runInBackground = async && hiveContext.sessionState.hiveThriftServerAsync val operation = new SparkExecuteStatementOperation(parentSession, statement, confOverlay, runInBackground)(hiveContext, sessionToActivePool) handleToOperation.put(operation.getHandle, operation) diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 989e68aebed9b..49fd19873017d 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -39,7 +39,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { private val originalLocale = Locale.getDefault private val originalColumnBatchSize = TestHive.conf.columnBatchSize private val originalInMemoryPartitionPruning = TestHive.conf.inMemoryPartitionPruning - private val originalConvertMetastoreOrc = TestHive.convertMetastoreOrc + private val originalConvertMetastoreOrc = TestHive.sessionState.convertMetastoreOrc def testCases: Seq[(String, File)] = { hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) @@ -47,7 +47,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { override def beforeAll() { super.beforeAll() - TestHive.cacheTables = true + TestHive.setCacheTables(true) // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) // Add Locale setting @@ -66,7 +66,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { override def afterAll() { try { - TestHive.cacheTables = false + TestHive.setCacheTables(false) TimeZone.setDefault(originalTimeZone) Locale.setDefault(originalLocale) TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE, originalColumnBatchSize) diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala index d0b4cbe401eb3..de592f8d937dd 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala @@ -38,7 +38,8 @@ class HiveWindowFunctionQuerySuite extends HiveComparisonTest with BeforeAndAfte private val testTempDir = Utils.createTempDir() override def beforeAll() { - TestHive.cacheTables = true + super.beforeAll() + TestHive.setCacheTables(true) // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) // Add Locale setting @@ -100,11 +101,14 @@ class HiveWindowFunctionQuerySuite extends HiveComparisonTest with BeforeAndAfte } override def afterAll() { - TestHive.cacheTables = false - TimeZone.setDefault(originalTimeZone) - Locale.setDefault(originalLocale) - TestHive.reset() - super.afterAll() + try { + TestHive.setCacheTables(false) + TimeZone.setDefault(originalTimeZone) + Locale.setDefault(originalLocale) + TestHive.reset() + } finally { + super.afterAll() + } } ///////////////////////////////////////////////////////////////////////////// @@ -773,7 +777,8 @@ class HiveWindowFunctionQueryFileSuite private val testTempDir = Utils.createTempDir() override def beforeAll() { - TestHive.cacheTables = true + super.beforeAll() + TestHive.setCacheTables(true) // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) // Add Locale setting @@ -790,10 +795,14 @@ class HiveWindowFunctionQueryFileSuite } override def afterAll() { - TestHive.cacheTables = false - TimeZone.setDefault(originalTimeZone) - Locale.setDefault(originalLocale) - TestHive.reset() + try { + TestHive.setCacheTables(false) + TimeZone.setDefault(originalTimeZone) + Locale.setDefault(originalLocale) + TestHive.reset() + } finally { + super.afterAll() + } } override def blackList: Seq[String] = Seq( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index b26a9ab699be1..b2ce3e0df25b4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -22,7 +22,6 @@ import java.net.{URL, URLClassLoader} import java.nio.charset.StandardCharsets import java.sql.Timestamp import java.util.concurrent.TimeUnit -import java.util.regex.Pattern import scala.collection.JavaConverters._ import scala.collection.mutable.HashMap @@ -32,26 +31,18 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.common.`type`.HiveDecimal import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hadoop.hive.ql.parse.VariableSubstitution import org.apache.hadoop.hive.serde2.io.{DateWritable, TimestampWritable} import org.apache.hadoop.util.VersionInfo import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.ConfigEntry +import org.apache.spark.internal.config.CATALOG_IMPLEMENTATION import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Expression, LeafExpression} -import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.execution.command.{ExecutedCommand, SetCommand} import org.apache.spark.sql.hive.client._ -import org.apache.spark.sql.hive.execution.{AnalyzeTable, DescribeHiveTableCommand, HiveNativeCommand} -import org.apache.spark.sql.internal.{SharedState, SQLConf} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf._ import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils /** @@ -61,225 +52,45 @@ import org.apache.spark.util.Utils * @since 1.0.0 */ class HiveContext private[hive]( - @transient protected[hive] val hiveSharedState: HiveSharedState, - override val isRootContext: Boolean) - extends SQLContext(hiveSharedState, isRootContext) with Logging { + @transient private val sparkSession: SparkSession, + isRootContext: Boolean) + extends SQLContext(sparkSession, isRootContext) with Logging { self => def this(sc: SparkContext) = { - this(new HiveSharedState(sc), true) + this(new SparkSession(HiveContext.withHiveExternalCatalog(sc)), true) } def this(sc: JavaSparkContext) = this(sc.sc) - import org.apache.spark.sql.hive.HiveContext._ - - logDebug("create HiveContext") - /** * Returns a new HiveContext as new session, which will have separated SQLConf, UDF/UDAF, * temporary tables and SessionState, but sharing the same CacheManager, IsolatedClientLoader * and Hive client (both of execution and metadata) with existing HiveContext. */ override def newSession(): HiveContext = { - new HiveContext(hiveSharedState, isRootContext = false) - } - - @transient - protected[sql] override lazy val sessionState = new HiveSessionState(self) - - protected[hive] def hiveCatalog: HiveExternalCatalog = hiveSharedState.externalCatalog - protected[hive] def executionHive: HiveClientImpl = sessionState.executionHive - protected[hive] def metadataHive: HiveClient = sessionState.metadataHive - - /** - * When true, enables an experimental feature where metastore tables that use the parquet SerDe - * are automatically converted to use the Spark SQL parquet table scan, instead of the Hive - * SerDe. - */ - protected[sql] def convertMetastoreParquet: Boolean = getConf(CONVERT_METASTORE_PARQUET) - - /** - * When true, also tries to merge possibly different but compatible Parquet schemas in different - * Parquet data files. - * - * This configuration is only effective when "spark.sql.hive.convertMetastoreParquet" is true. - */ - protected[sql] def convertMetastoreParquetWithSchemaMerging: Boolean = - getConf(CONVERT_METASTORE_PARQUET_WITH_SCHEMA_MERGING) - - /** - * When true, enables an experimental feature where metastore tables that use the Orc SerDe - * are automatically converted to use the Spark SQL ORC table scan, instead of the Hive - * SerDe. - */ - protected[sql] def convertMetastoreOrc: Boolean = getConf(CONVERT_METASTORE_ORC) - - /** - * When true, a table created by a Hive CTAS statement (no USING clause) will be - * converted to a data source table, using the data source set by spark.sql.sources.default. - * The table in CTAS statement will be converted when it meets any of the following conditions: - * - The CTAS does not specify any of a SerDe (ROW FORMAT SERDE), a File Format (STORED AS), or - * a Storage Hanlder (STORED BY), and the value of hive.default.fileformat in hive-site.xml - * is either TextFile or SequenceFile. - * - The CTAS statement specifies TextFile (STORED AS TEXTFILE) as the file format and no SerDe - * is specified (no ROW FORMAT SERDE clause). - * - The CTAS statement specifies SequenceFile (STORED AS SEQUENCEFILE) as the file format - * and no SerDe is specified (no ROW FORMAT SERDE clause). - */ - protected[sql] def convertCTAS: Boolean = getConf(CONVERT_CTAS) - - /* - * hive thrift server use background spark sql thread pool to execute sql queries - */ - protected[hive] def hiveThriftServerAsync: Boolean = getConf(HIVE_THRIFT_SERVER_ASYNC) - - protected[hive] def hiveThriftServerSingleSession: Boolean = - sparkContext.conf.getBoolean("spark.sql.hive.thriftServer.singleSession", defaultValue = false) - - @transient - protected[sql] lazy val substitutor = new VariableSubstitution() - - /** - * Overrides default Hive configurations to avoid breaking changes to Spark SQL users. - * - allow SQL11 keywords to be used as identifiers - */ - private[sql] def defaultOverrides() = { - setConf(ConfVars.HIVE_SUPPORT_SQL11_RESERVED_KEYWORDS.varname, "false") - } - - defaultOverrides() - - protected[sql] override def parseSql(sql: String): LogicalPlan = { - executionHive.withHiveState { - super.parseSql(substitutor.substitute(sessionState.hiveconf, sql)) - } - } - - override protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution = - new this.QueryExecution(plan) - - /** - * Invalidate and refresh all the cached the metadata of the given table. For performance reasons, - * Spark SQL or the external data source library it uses might cache certain metadata about a - * table, such as the location of blocks. When those change outside of Spark SQL, users should - * call this function to invalidate the cache. - * - * @since 1.3.0 - */ - def refreshTable(tableName: String): Unit = { - val tableIdent = sessionState.sqlParser.parseTableIdentifier(tableName) - sessionState.catalog.refreshTable(tableIdent) - } - - protected[hive] def invalidateTable(tableName: String): Unit = { - val tableIdent = sessionState.sqlParser.parseTableIdentifier(tableName) - sessionState.catalog.invalidateTable(tableIdent) - } - - /** - * Analyzes the given table in the current database to generate statistics, which will be - * used in query optimizations. - * - * Right now, it only supports Hive tables and it only updates the size of a Hive table - * in the Hive metastore. - * - * @since 1.2.0 - */ - def analyze(tableName: String) { - AnalyzeTable(tableName).run(self) - } - - override def setConf(key: String, value: String): Unit = { - super.setConf(key, value) - executionHive.runSqlHive(s"SET $key=$value") - metadataHive.runSqlHive(s"SET $key=$value") - // If users put any Spark SQL setting in the spark conf (e.g. spark-defaults.conf), - // this setConf will be called in the constructor of the SQLContext. - // Also, calling hiveconf will create a default session containing a HiveConf, which - // will interfer with the creation of executionHive (which is a lazy val). So, - // we put hiveconf.set at the end of this method. - sessionState.hiveconf.set(key, value) + new HiveContext(sparkSession.newSession(), isRootContext = false) } - override private[sql] def setConf[T](entry: ConfigEntry[T], value: T): Unit = { - setConf(entry.key, entry.stringConverter(value)) + protected[sql] override def sessionState: HiveSessionState = { + sparkSession.sessionState.asInstanceOf[HiveSessionState] } - private def functionOrMacroDDLPattern(command: String) = Pattern.compile( - ".*(create|drop)\\s+(temporary\\s+)?(function|macro).+", Pattern.DOTALL).matcher(command) - - protected[hive] def runSqlHive(sql: String): Seq[String] = { - val command = sql.trim.toLowerCase - if (functionOrMacroDDLPattern(command).matches()) { - executionHive.runSqlHive(sql) - } else if (command.startsWith("set")) { - metadataHive.runSqlHive(sql) - executionHive.runSqlHive(sql) - } else { - metadataHive.runSqlHive(sql) - } + protected[sql] override def sharedState: HiveSharedState = { + sparkSession.sharedState.asInstanceOf[HiveSharedState] } - /** - * Executes a SQL query without parsing it, but instead passing it directly to Hive. - * This is currently only used for DDLs and will be removed as soon as Spark can parse - * all supported Hive DDLs itself. - */ - protected[sql] override def runNativeSql(sqlText: String): Seq[Row] = { - runSqlHive(sqlText).map { s => Row(s) } - } +} - /** Extends QueryExecution with hive specific features. */ - protected[sql] class QueryExecution(logicalPlan: LogicalPlan) - extends org.apache.spark.sql.execution.QueryExecution(this, logicalPlan) { - - /** - * Returns the result as a hive compatible sequence of strings. For native commands, the - * execution is simply passed back to Hive. - */ - def stringResult(): Seq[String] = executedPlan match { - case ExecutedCommand(desc: DescribeHiveTableCommand) => - // If it is a describe command for a Hive table, we want to have the output format - // be similar with Hive. - desc.run(self).map { - case Row(name: String, dataType: String, comment) => - Seq(name, dataType, - Option(comment.asInstanceOf[String]).getOrElse("")) - .map(s => String.format(s"%-20s", s)) - .mkString("\t") - } - case command: ExecutedCommand => - command.executeCollect().map(_.getString(0)) - - case other => - val result: Seq[Seq[Any]] = other.executeCollectPublic().map(_.toSeq).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. - result.map(_.zip(types).map(HiveContext.toHiveString)).map(_.mkString("\t")).toSeq - } - override def simpleString: String = - logical match { - case _: HiveNativeCommand => "" - case _: SetCommand => "" - case _ => super.simpleString - } - } +private[hive] object HiveContext extends Logging { - protected[sql] override def addJar(path: String): Unit = { - // Add jar to Hive and classloader - executionHive.addJar(path) - metadataHive.addJar(path) - Thread.currentThread().setContextClassLoader(executionHive.clientLoader.classLoader) - super.addJar(path) + def withHiveExternalCatalog(sc: SparkContext): SparkContext = { + sc.conf.set(CATALOG_IMPLEMENTATION.key, "hive") + sc } -} - -private[hive] object HiveContext extends Logging { /** The version of hive used internally by Spark SQL. */ val hiveExecutionVersion: String = "1.2.1" diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 753950ff844c9..33a926e4d2551 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -115,17 +115,16 @@ private[hive] object HiveSerDe { * This is still used for things like creating data source tables, but in the future will be * cleaned up to integrate more nicely with [[HiveExternalCatalog]]. */ -private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveContext) - extends Logging { - - val conf = hive.conf +private[hive] class HiveMetastoreCatalog(hive: SQLContext) extends Logging { + private val conf = hive.conf + private val sessionState = hive.sessionState.asInstanceOf[HiveSessionState] + private val client = hive.sharedState.asInstanceOf[HiveSharedState].metadataHive + private val hiveconf = sessionState.hiveconf /** A fully qualified identifier for a table (i.e., database.tableName) */ case class QualifiedTableName(database: String, name: String) - private def getCurrentDatabase: String = { - hive.sessionState.catalog.getCurrentDatabase - } + private def getCurrentDatabase: String = hive.sessionState.catalog.getCurrentDatabase def getQualifiedTableName(tableIdent: TableIdentifier): QualifiedTableName = { QualifiedTableName( @@ -298,7 +297,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte CatalogTableType.MANAGED_TABLE } - val maybeSerDe = HiveSerDe.sourceToSerDe(provider, hive.sessionState.hiveconf) + val maybeSerDe = HiveSerDe.sourceToSerDe(provider, hiveconf) val dataSource = DataSource( hive, @@ -600,14 +599,14 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte object ParquetConversions extends Rule[LogicalPlan] { private def shouldConvertMetastoreParquet(relation: MetastoreRelation): Boolean = { relation.tableDesc.getSerdeClassName.toLowerCase.contains("parquet") && - hive.convertMetastoreParquet + sessionState.convertMetastoreParquet } private def convertToParquetRelation(relation: MetastoreRelation): LogicalRelation = { val defaultSource = new ParquetDefaultSource() val fileFormatClass = classOf[ParquetDefaultSource] - val mergeSchema = hive.convertMetastoreParquetWithSchemaMerging + val mergeSchema = sessionState.convertMetastoreParquetWithSchemaMerging val options = Map( ParquetRelation.MERGE_SCHEMA -> mergeSchema.toString, ParquetRelation.METASTORE_TABLE_NAME -> TableIdentifier( @@ -652,7 +651,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte object OrcConversions extends Rule[LogicalPlan] { private def shouldConvertMetastoreOrc(relation: MetastoreRelation): Boolean = { relation.tableDesc.getSerdeClassName.toLowerCase.contains("orc") && - hive.convertMetastoreOrc + sessionState.convertMetastoreOrc } private def convertToOrcRelation(relation: MetastoreRelation): LogicalRelation = { @@ -727,7 +726,7 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte val desc = table.copy(schema = schema) - if (hive.convertCTAS && table.storage.serde.isEmpty) { + if (sessionState.convertCTAS && table.storage.serde.isEmpty) { // Do the conversion when spark.sql.hive.convertCTAS is true and the query // does not specify any storage format (file format and storage handler). if (table.identifier.database.isDefined) { @@ -815,14 +814,13 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte * the information from the metastore. */ class MetaStoreFileCatalog( - hive: HiveContext, + ctx: SQLContext, paths: Seq[Path], partitionSpecFromHive: PartitionSpec) - extends HDFSFileCatalog(hive, Map.empty, paths, Some(partitionSpecFromHive.partitionColumns)) { - + extends HDFSFileCatalog(ctx, Map.empty, paths, Some(partitionSpecFromHive.partitionColumns)) { override def getStatus(path: Path): Array[FileStatus] = { - val fs = path.getFileSystem(hive.sparkContext.hadoopConfiguration) + val fs = path.getFileSystem(ctx.sparkContext.hadoopConfiguration) fs.listStatus(path) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQueryExecution.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQueryExecution.scala new file mode 100644 index 0000000000000..1c1bfb610c29e --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQueryExecution.scala @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.execution.command.{ExecutedCommand, SetCommand} +import org.apache.spark.sql.hive.execution.{DescribeHiveTableCommand, HiveNativeCommand} + + +/** + * A [[QueryExecution]] with hive specific features. + */ +protected[hive] class HiveQueryExecution(ctx: SQLContext, logicalPlan: LogicalPlan) + extends QueryExecution(ctx, logicalPlan) { + + /** + * Returns the result as a hive compatible sequence of strings. For native commands, the + * execution is simply passed back to Hive. + */ + def stringResult(): Seq[String] = executedPlan match { + case ExecutedCommand(desc: DescribeHiveTableCommand) => + // If it is a describe command for a Hive table, we want to have the output format + // be similar with Hive. + desc.run(ctx).map { + case Row(name: String, dataType: String, comment) => + Seq(name, dataType, + Option(comment.asInstanceOf[String]).getOrElse("")) + .map(s => String.format(s"%-20s", s)) + .mkString("\t") + } + case command: ExecutedCommand => + command.executeCollect().map(_.getString(0)) + + case other => + val result: Seq[Seq[Any]] = other.executeCollectPublic().map(_.toSeq).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. + result.map(_.zip(types).map(HiveContext.toHiveString)).map(_.mkString("\t")).toSeq + } + + override def simpleString: String = + logical match { + case _: HiveNativeCommand => "" + case _: SetCommand => "" + case _ => super.simpleString + } + +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index f91393fc76324..4f9513389c8c2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.exec.{UDAF, UDF} import org.apache.hadoop.hive.ql.exec.{FunctionRegistry => HiveFunctionRegistry} import org.apache.hadoop.hive.ql.udf.generic.{AbstractGenericUDAFResolver, GenericUDF, GenericUDTF} +import org.apache.spark.sql.{AnalysisException, SQLContext} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder @@ -33,7 +34,6 @@ import org.apache.spark.sql.catalyst.catalog.{FunctionResourceLoader, SessionCat import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.execution.datasources.BucketSpec import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper import org.apache.spark.sql.hive.client.HiveClient @@ -45,10 +45,11 @@ import org.apache.spark.util.Utils private[sql] class HiveSessionCatalog( externalCatalog: HiveExternalCatalog, client: HiveClient, - context: HiveContext, + context: SQLContext, functionResourceLoader: FunctionResourceLoader, functionRegistry: FunctionRegistry, - conf: SQLConf) + conf: SQLConf, + hiveconf: HiveConf) extends SessionCatalog(externalCatalog, functionResourceLoader, functionRegistry, conf) { override def setCurrentDatabase(db: String): Unit = { @@ -75,7 +76,7 @@ private[sql] class HiveSessionCatalog( // ---------------------------------------------------------------- override def getDefaultDBPath(db: String): String = { - val defaultPath = context.sessionState.hiveconf.getVar(HiveConf.ConfVars.METASTOREWAREHOUSE) + val defaultPath = hiveconf.getVar(HiveConf.ConfVars.METASTOREWAREHOUSE) new Path(new Path(defaultPath), db + ".db").toString } @@ -83,7 +84,7 @@ private[sql] class HiveSessionCatalog( // essentially a cache for metastore tables. However, it relies on a lot of session-specific // things so it would be a lot of work to split its functionality between HiveSessionCatalog // and HiveCatalog. We should still do it at some point... - private val metastoreCatalog = new HiveMetastoreCatalog(client, context) + private val metastoreCatalog = new HiveMetastoreCatalog(context) val ParquetConversions: Rule[LogicalPlan] = metastoreCatalog.ParquetConversions val OrcConversions: Rule[LogicalPlan] = metastoreCatalog.OrcConversions diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala index 2b848524f3e90..09297c27dc5bc 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala @@ -17,62 +17,80 @@ package org.apache.spark.sql.hive +import java.util.regex.Pattern + import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hadoop.hive.ql.parse.VariableSubstitution import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.Analyzer import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlanner import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.hive.client.{HiveClient, HiveClientImpl} -import org.apache.spark.sql.hive.execution.HiveSqlParser +import org.apache.spark.sql.hive.execution.{AnalyzeTable, HiveSqlParser} import org.apache.spark.sql.internal.{SessionState, SQLConf} /** * A class that holds all session-specific state in a given [[HiveContext]]. */ -private[hive] class HiveSessionState(ctx: HiveContext) extends SessionState(ctx) { +private[hive] class HiveSessionState(ctx: SQLContext) extends SessionState(ctx) { - /** - * SQLConf and HiveConf contracts: - * - * 1. create a new o.a.h.hive.ql.session.SessionState for each [[HiveContext]] - * 2. when the Hive session is first initialized, params in HiveConf will get picked up by the - * SQLConf. Additionally, any properties set by set() or a SET command inside sql() will be - * set in the SQLConf *as well as* in the HiveConf. - */ - lazy val hiveconf: HiveConf = { - val c = ctx.executionHive.conf - ctx.setConf(c.getAllProperties) - c - } + self => + + private lazy val sharedState: HiveSharedState = ctx.sharedState.asInstanceOf[HiveSharedState] /** * A Hive client used for execution. */ - val executionHive: HiveClientImpl = ctx.hiveSharedState.executionHive.newSession() + lazy val executionHive: HiveClientImpl = sharedState.executionHive.newSession() /** * A Hive client used for interacting with the metastore. */ - val metadataHive: HiveClient = ctx.hiveSharedState.metadataHive.newSession() + lazy val metadataHive: HiveClient = sharedState.metadataHive.newSession() + + /** + * A Hive helper class for substituting variables in a SQL statement. + */ + lazy val substitutor = new VariableSubstitution override lazy val conf: SQLConf = new SQLConf { override def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE, false) } + + /** + * SQLConf and HiveConf contracts: + * + * 1. create a new o.a.h.hive.ql.session.SessionState for each HiveContext + * 2. when the Hive session is first initialized, params in HiveConf will get picked up by the + * SQLConf. Additionally, any properties set by set() or a SET command inside sql() will be + * set in the SQLConf *as well as* in the HiveConf. + */ + lazy val hiveconf: HiveConf = { + val c = executionHive.conf + conf.setConf(c.getAllProperties) + c + } + + setDefaultOverrideConfs() + /** * Internal catalog for managing table and database states. */ override lazy val catalog = { new HiveSessionCatalog( - ctx.hiveCatalog, - ctx.metadataHive, + sharedState.externalCatalog, + metadataHive, ctx, ctx.functionResourceLoader, functionRegistry, - conf) + conf, + hiveconf) } /** @@ -96,7 +114,7 @@ private[hive] class HiveSessionState(ctx: HiveContext) extends SessionState(ctx) /** * Parser for HiveQl query texts. */ - override lazy val sqlParser: ParserInterface = new HiveSqlParser(hiveconf) + override lazy val sqlParser: ParserInterface = new HiveSqlParser(substitutor, hiveconf) /** * Planner that takes into account Hive-specific strategies. @@ -104,13 +122,14 @@ private[hive] class HiveSessionState(ctx: HiveContext) extends SessionState(ctx) override def planner: SparkPlanner = { new SparkPlanner(ctx.sparkContext, conf, experimentalMethods.extraStrategies) with HiveStrategies { - override val hiveContext = ctx + override val context: SQLContext = ctx + override val hiveconf: HiveConf = self.hiveconf override def strategies: Seq[Strategy] = { experimentalMethods.extraStrategies ++ Seq( FileSourceStrategy, DataSourceStrategy, - HiveCommandStrategy(ctx), + HiveCommandStrategy, HiveDDLStrategy, DDLStrategy, SpecialLimits, @@ -130,4 +149,119 @@ private[hive] class HiveSessionState(ctx: HiveContext) extends SessionState(ctx) } } + + // ------------------------------------------------------ + // Helper methods, partially leftover from pre-2.0 days + // ------------------------------------------------------ + + override def executePlan(plan: LogicalPlan): HiveQueryExecution = { + new HiveQueryExecution(ctx, plan) + } + + /** + * Overrides default Hive configurations to avoid breaking changes to Spark SQL users. + * - allow SQL11 keywords to be used as identifiers + */ + def setDefaultOverrideConfs(): Unit = { + setConf(ConfVars.HIVE_SUPPORT_SQL11_RESERVED_KEYWORDS.varname, "false") + } + + override def setConf(key: String, value: String): Unit = { + super.setConf(key, value) + executionHive.runSqlHive(s"SET $key=$value") + metadataHive.runSqlHive(s"SET $key=$value") + hiveconf.set(key, value) + } + + override def addJar(path: String): Unit = { + super.addJar(path) + executionHive.addJar(path) + metadataHive.addJar(path) + Thread.currentThread().setContextClassLoader(executionHive.clientLoader.classLoader) + } + + /** + * Analyzes the given table in the current database to generate statistics, which will be + * used in query optimizations. + * + * Right now, it only supports Hive tables and it only updates the size of a Hive table + * in the Hive metastore. + */ + override def analyze(tableName: String): Unit = { + AnalyzeTable(tableName).run(ctx) + } + + /** + * Execute a SQL statement by passing the query text directly to Hive. + */ + override def runNativeSql(sql: String): Seq[String] = { + val command = sql.trim.toLowerCase + val functionOrMacroDDLPattern = Pattern.compile( + ".*(create|drop)\\s+(temporary\\s+)?(function|macro).+", Pattern.DOTALL) + if (functionOrMacroDDLPattern.matcher(command).matches()) { + executionHive.runSqlHive(sql) + } else if (command.startsWith("set")) { + metadataHive.runSqlHive(sql) + executionHive.runSqlHive(sql) + } else { + metadataHive.runSqlHive(sql) + } + } + + /** + * When true, enables an experimental feature where metastore tables that use the parquet SerDe + * are automatically converted to use the Spark SQL parquet table scan, instead of the Hive + * SerDe. + */ + def convertMetastoreParquet: Boolean = { + conf.getConf(HiveContext.CONVERT_METASTORE_PARQUET) + } + + /** + * When true, also tries to merge possibly different but compatible Parquet schemas in different + * Parquet data files. + * + * This configuration is only effective when "spark.sql.hive.convertMetastoreParquet" is true. + */ + def convertMetastoreParquetWithSchemaMerging: Boolean = { + conf.getConf(HiveContext.CONVERT_METASTORE_PARQUET_WITH_SCHEMA_MERGING) + } + + /** + * When true, enables an experimental feature where metastore tables that use the Orc SerDe + * are automatically converted to use the Spark SQL ORC table scan, instead of the Hive + * SerDe. + */ + def convertMetastoreOrc: Boolean = { + conf.getConf(HiveContext.CONVERT_METASTORE_ORC) + } + + /** + * When true, a table created by a Hive CTAS statement (no USING clause) will be + * converted to a data source table, using the data source set by spark.sql.sources.default. + * The table in CTAS statement will be converted when it meets any of the following conditions: + * - The CTAS does not specify any of a SerDe (ROW FORMAT SERDE), a File Format (STORED AS), or + * a Storage Hanlder (STORED BY), and the value of hive.default.fileformat in hive-site.xml + * is either TextFile or SequenceFile. + * - The CTAS statement specifies TextFile (STORED AS TEXTFILE) as the file format and no SerDe + * is specified (no ROW FORMAT SERDE clause). + * - The CTAS statement specifies SequenceFile (STORED AS SEQUENCEFILE) as the file format + * and no SerDe is specified (no ROW FORMAT SERDE clause). + */ + def convertCTAS: Boolean = { + conf.getConf(HiveContext.CONVERT_CTAS) + } + + /** + * When true, Hive Thrift server will execute SQL queries asynchronously using a thread pool." + */ + def hiveThriftServerAsync: Boolean = { + conf.getConf(HiveContext.HIVE_THRIFT_SERVER_ASYNC) + } + + def hiveThriftServerSingleSession: Boolean = { + ctx.sparkContext.conf.getBoolean( + "spark.sql.hive.thriftServer.singleSession", defaultValue = false) + } + } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 010361a32eb34..bbdcc8c6c2fff 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive +import org.apache.hadoop.hive.conf.HiveConf + import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ @@ -31,12 +33,13 @@ private[hive] trait HiveStrategies { // Possibly being too clever with types here... or not clever enough. self: SparkPlanner => - val hiveContext: HiveContext + val context: SQLContext + val hiveconf: HiveConf object Scripts extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.ScriptTransformation(input, script, output, child, schema: HiveScriptIOSchema) => - ScriptTransformation(input, script, output, planLater(child), schema)(hiveContext) :: Nil + ScriptTransformation(input, script, output, planLater(child), schema)(hiveconf) :: Nil case _ => Nil } } @@ -74,7 +77,7 @@ private[hive] trait HiveStrategies { projectList, otherPredicates, identity[Seq[Expression]], - HiveTableScan(_, relation, pruningPredicates)(hiveContext)) :: Nil + HiveTableScan(_, relation, pruningPredicates)(context, hiveconf)) :: Nil case _ => Nil } @@ -103,7 +106,7 @@ private[hive] trait HiveStrategies { } } - case class HiveCommandStrategy(context: HiveContext) extends Strategy { + case object HiveCommandStrategy extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case describe: DescribeCommand => ExecutedCommand( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index 98a427380d97b..6a20d7c25b682 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -37,6 +37,7 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.DateTimeUtils @@ -61,8 +62,8 @@ private[hive] class HadoopTableReader( @transient private val attributes: Seq[Attribute], @transient private val relation: MetastoreRelation, - @transient private val sc: HiveContext, - hiveExtraConf: HiveConf) + @transient private val sc: SQLContext, + hiveconf: HiveConf) extends TableReader with Logging { // Hadoop honors "mapred.map.tasks" as hint, but will ignore when mapred.job.tracker is "local". @@ -72,14 +73,12 @@ class HadoopTableReader( private val _minSplitsPerRDD = if (sc.sparkContext.isLocal) { 0 // will splitted based on block by default. } else { - math.max( - sc.sessionState.hiveconf.getInt("mapred.map.tasks", 1), - sc.sparkContext.defaultMinPartitions) + math.max(hiveconf.getInt("mapred.map.tasks", 1), sc.sparkContext.defaultMinPartitions) } - SparkHadoopUtil.get.appendS3AndSparkHadoopConfigurations(sc.sparkContext.conf, hiveExtraConf) + SparkHadoopUtil.get.appendS3AndSparkHadoopConfigurations(sc.sparkContext.conf, hiveconf) private val _broadcastedHiveConf = - sc.sparkContext.broadcast(new SerializableConfiguration(hiveExtraConf)) + sc.sparkContext.broadcast(new SerializableConfiguration(hiveconf)) override def makeRDDForTable(hiveTable: HiveTable): RDD[InternalRow] = makeRDDForTable( @@ -164,7 +163,7 @@ class HadoopTableReader( case (partition, partDeserializer) => def updateExistPathSetByPathPattern(pathPatternStr: String) { val pathPattern = new Path(pathPatternStr) - val fs = pathPattern.getFileSystem(sc.sessionState.hiveconf) + val fs = pathPattern.getFileSystem(hiveconf) val matches = fs.globStatus(pathPattern) matches.foreach(fileStatus => existPathSet += fileStatus.getPath.toString) } @@ -261,7 +260,7 @@ class HadoopTableReader( private def applyFilterIfNeeded(path: Path, filterOpt: Option[PathFilter]): String = { filterOpt match { case Some(filter) => - val fs = path.getFileSystem(sc.sessionState.hiveconf) + val fs = path.getFileSystem(hiveconf) val filteredFiles = fs.listStatus(path, filter).map(_.getPath.toString) filteredFiles.mkString(",") case None => path.toString diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 2a1fff92b570a..69f7dbf6ceebf 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -151,6 +151,8 @@ private[hive] class HiveClientImpl( } /** Returns the configuration for the current session. */ + // TODO: We should not use it because HiveSessionState has a hiveconf + // for the current Session. def conf: HiveConf = SessionState.get().getConf override def getConf(key: String, defaultValue: String): String = { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala index 29f7dc2997d26..ceb7f3b890949 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala @@ -43,7 +43,6 @@ case class CreateTableAsSelect( override def children: Seq[LogicalPlan] = Seq(query) override def run(sqlContext: SQLContext): Seq[Row] = { - val hiveContext = sqlContext.asInstanceOf[HiveContext] lazy val metastoreRelation: MetastoreRelation = { import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe @@ -69,24 +68,24 @@ case class CreateTableAsSelect( withFormat } - hiveContext.sessionState.catalog.createTable(withSchema, ignoreIfExists = false) + sqlContext.sessionState.catalog.createTable(withSchema, ignoreIfExists = false) // Get the Metastore Relation - hiveContext.sessionState.catalog.lookupRelation(tableIdentifier) match { + sqlContext.sessionState.catalog.lookupRelation(tableIdentifier) match { case r: MetastoreRelation => r } } // TODO ideally, we should get the output data ready first and then // add the relation into catalog, just in case of failure occurs while data // processing. - if (hiveContext.sessionState.catalog.tableExists(tableIdentifier)) { + if (sqlContext.sessionState.catalog.tableExists(tableIdentifier)) { if (allowExisting) { // table already exists, will do nothing, to keep consistent with Hive } else { throw new AnalysisException(s"$tableIdentifier already exists.") } } else { - hiveContext.executePlan(InsertIntoTable(metastoreRelation, Map(), query, true, false)).toRdd + sqlContext.executePlan(InsertIntoTable(metastoreRelation, Map(), query, true, false)).toRdd } Seq.empty[Row] diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateViewAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateViewAsSelect.scala index 33cd8b44805b8..1e234d8508b40 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateViewAsSelect.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateViewAsSelect.scala @@ -20,12 +20,11 @@ package org.apache.spark.sql.hive.execution import scala.util.control.NonFatal import org.apache.spark.sql.{AnalysisException, Row, SQLContext} -import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogColumn, CatalogTable} import org.apache.spark.sql.catalyst.expressions.Alias import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} import org.apache.spark.sql.execution.command.RunnableCommand -import org.apache.spark.sql.hive.{ HiveContext, HiveMetastoreTypes, SQLBuilder} +import org.apache.spark.sql.hive.{HiveMetastoreTypes, HiveSessionState, SQLBuilder} /** * Create Hive view on non-hive-compatible tables by specifying schema ourselves instead of @@ -47,16 +46,16 @@ private[hive] case class CreateViewAsSelect( private val tableIdentifier = tableDesc.identifier override def run(sqlContext: SQLContext): Seq[Row] = { - val hiveContext = sqlContext.asInstanceOf[HiveContext] + val sessionState = sqlContext.sessionState.asInstanceOf[HiveSessionState] - hiveContext.sessionState.catalog.tableExists(tableIdentifier) match { + sessionState.catalog.tableExists(tableIdentifier) match { case true if allowExisting => // Handles `CREATE VIEW IF NOT EXISTS v0 AS SELECT ...`. Does nothing when the target view // already exists. case true if orReplace => // Handles `CREATE OR REPLACE VIEW v0 AS SELECT ...` - hiveContext.metadataHive.alertView(prepareTable(sqlContext)) + sessionState.metadataHive.alertView(prepareTable(sqlContext)) case true => // Handles `CREATE VIEW v0 AS SELECT ...`. Throws exception when the target view already @@ -66,7 +65,7 @@ private[hive] case class CreateViewAsSelect( "CREATE OR REPLACE VIEW AS") case false => - hiveContext.metadataHive.createView(prepareTable(sqlContext)) + sessionState.metadataHive.createView(prepareTable(sqlContext)) } Seq.empty[Row] diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala index 9bb971992d0d1..8c1f4a8dc5139 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.execution.command.RunnableCommand -import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.HiveSessionState import org.apache.spark.sql.types.StringType private[hive] @@ -29,6 +29,8 @@ case class HiveNativeCommand(sql: String) extends RunnableCommand { override def output: Seq[AttributeReference] = Seq(AttributeReference("result", StringType, nullable = false)()) - override def run(sqlContext: SQLContext): Seq[Row] = - sqlContext.asInstanceOf[HiveContext].runSqlHive(sql).map(Row(_)) + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.sessionState.asInstanceOf[HiveSessionState].runNativeSql(sql).map(Row(_)) + } + } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala index d5d3ee43d7e82..4ff02cdbd0b39 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala @@ -21,8 +21,7 @@ import scala.collection.JavaConverters._ import org.antlr.v4.runtime.{ParserRuleContext, Token} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hadoop.hive.ql.parse.EximUtil -import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hadoop.hive.ql.parse.{EximUtil, VariableSubstitution} import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe @@ -39,11 +38,19 @@ import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper /** * Concrete parser for HiveQl statements. */ -class HiveSqlParser(hiveConf: HiveConf) extends AbstractSqlParser { - val astBuilder = new HiveSqlAstBuilder(hiveConf) +class HiveSqlParser( + substitutor: VariableSubstitution, + hiveconf: HiveConf) + extends AbstractSqlParser { - override protected def nativeCommand(sqlText: String): LogicalPlan = { - HiveNativeCommand(sqlText) + val astBuilder = new HiveSqlAstBuilder(hiveconf) + + protected override def parse[T](command: String)(toResult: SqlBaseParser => T): T = { + super.parse(substitutor.substitute(hiveconf, command))(toResult) + } + + protected override def nativeCommand(sqlText: String): LogicalPlan = { + HiveNativeCommand(substitutor.substitute(hiveconf, sqlText)) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala index 3c46b836dcec3..9a834660f953f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala @@ -27,6 +27,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.Object import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution._ @@ -47,7 +48,8 @@ case class HiveTableScan( requestedAttributes: Seq[Attribute], relation: MetastoreRelation, partitionPruningPred: Seq[Expression])( - @transient val context: HiveContext) + @transient val context: SQLContext, + @transient val hiveconf: HiveConf) extends LeafNode { require(partitionPruningPred.isEmpty || relation.hiveQlTable.isPartitioned, @@ -75,7 +77,7 @@ case class HiveTableScan( // Create a local copy of hiveconf,so that scan specific modifications should not impact // other queries @transient - private[this] val hiveExtraConf = new HiveConf(context.sessionState.hiveconf) + private[this] val hiveExtraConf = new HiveConf(hiveconf) // append columns ids and names before broadcast addColumnMetadataToConf(hiveExtraConf) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index ed538630d24a8..e614daadf3918 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -43,9 +43,10 @@ case class InsertIntoHiveTable( overwrite: Boolean, ifNotExists: Boolean) extends UnaryNode { - @transient val sc: HiveContext = sqlContext.asInstanceOf[HiveContext] - @transient private lazy val hiveContext = new Context(sc.sessionState.hiveconf) - @transient private lazy val client = sc.metadataHive + @transient private val sessionState = sqlContext.sessionState.asInstanceOf[HiveSessionState] + @transient private val client = sessionState.metadataHive + @transient private val hiveconf = sessionState.hiveconf + @transient private lazy val hiveContext = new Context(hiveconf) def output: Seq[Attribute] = Seq.empty @@ -67,7 +68,7 @@ case class InsertIntoHiveTable( SparkHiveWriterContainer.createPathFromString(fileSinkConf.getDirName, conf.value)) log.debug("Saving as hadoop file of type " + valueClass.getSimpleName) writerContainer.driverSideSetup() - sc.sparkContext.runJob(rdd, writerContainer.writeToFile _) + sqlContext.sparkContext.runJob(rdd, writerContainer.writeToFile _) writerContainer.commitJob() } @@ -86,17 +87,17 @@ case class InsertIntoHiveTable( val tableLocation = table.hiveQlTable.getDataLocation val tmpLocation = hiveContext.getExternalTmpPath(tableLocation) val fileSinkConf = new FileSinkDesc(tmpLocation.toString, tableDesc, false) - val isCompressed = sc.sessionState.hiveconf.getBoolean( + val isCompressed = hiveconf.getBoolean( ConfVars.COMPRESSRESULT.varname, ConfVars.COMPRESSRESULT.defaultBoolVal) if (isCompressed) { // Please note that isCompressed, "mapred.output.compress", "mapred.output.compression.codec", // and "mapred.output.compression.type" have no impact on ORC because it uses table properties // to store compression information. - sc.sessionState.hiveconf.set("mapred.output.compress", "true") + hiveconf.set("mapred.output.compress", "true") fileSinkConf.setCompressed(true) - fileSinkConf.setCompressCodec(sc.sessionState.hiveconf.get("mapred.output.compression.codec")) - fileSinkConf.setCompressType(sc.sessionState.hiveconf.get("mapred.output.compression.type")) + fileSinkConf.setCompressCodec(hiveconf.get("mapred.output.compression.codec")) + fileSinkConf.setCompressType(hiveconf.get("mapred.output.compression.type")) } val numDynamicPartitions = partition.values.count(_.isEmpty) @@ -113,12 +114,12 @@ case class InsertIntoHiveTable( // Validate partition spec if there exist any dynamic partitions if (numDynamicPartitions > 0) { // Report error if dynamic partitioning is not enabled - if (!sc.sessionState.hiveconf.getBoolVar(HiveConf.ConfVars.DYNAMICPARTITIONING)) { + if (!hiveconf.getBoolVar(HiveConf.ConfVars.DYNAMICPARTITIONING)) { throw new SparkException(ErrorMsg.DYNAMIC_PARTITION_DISABLED.getMsg) } // Report error if dynamic partition strict mode is on but no static partition is found - if (numStaticPartitions == 0 && sc.sessionState.hiveconf.getVar( + if (numStaticPartitions == 0 && hiveconf.getVar( HiveConf.ConfVars.DYNAMICPARTITIONINGMODE).equalsIgnoreCase("strict")) { throw new SparkException(ErrorMsg.DYNAMIC_PARTITION_STRICT_MODE.getMsg) } @@ -130,7 +131,7 @@ case class InsertIntoHiveTable( } } - val jobConf = new JobConf(sc.sessionState.hiveconf) + val jobConf = new JobConf(hiveconf) val jobConfSer = new SerializableJobConf(jobConf) // When speculation is on and output committer class name contains "Direct", we should warn diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index ea48b0e5c26de..2f7cec354d84f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -26,6 +26,7 @@ import scala.collection.JavaConverters._ import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.exec.{RecordReader, RecordWriter} import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.AbstractSerDe @@ -39,7 +40,7 @@ import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.ScriptInputOutputSchema import org.apache.spark.sql.execution._ -import org.apache.spark.sql.hive.{HiveContext, HiveInspectors} +import org.apache.spark.sql.hive.HiveInspectors import org.apache.spark.sql.hive.HiveShim._ import org.apache.spark.sql.types.DataType import org.apache.spark.util.{CircularBuffer, RedirectThread, SerializableConfiguration, Utils} @@ -57,14 +58,14 @@ case class ScriptTransformation( script: String, output: Seq[Attribute], child: SparkPlan, - ioschema: HiveScriptIOSchema)(@transient private val sc: HiveContext) + ioschema: HiveScriptIOSchema)(@transient private val hiveconf: HiveConf) extends UnaryNode { - override protected def otherCopyArgs: Seq[HiveContext] = sc :: Nil + override protected def otherCopyArgs: Seq[HiveConf] = hiveconf :: Nil override def producedAttributes: AttributeSet = outputSet -- inputSet - private val serializedHiveConf = new SerializableConfiguration(sc.sessionState.hiveconf) + private val serializedHiveConf = new SerializableConfiguration(hiveconf) protected override def doExecute(): RDD[InternalRow] = { def processIterator(inputIterator: Iterator[InternalRow]): Iterator[InternalRow] = { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 7a2b60dde5a3d..b5ee9a62954ce 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.execution.datasources.{BucketSpec, DataSource, HadoopFsRelation, LogicalRelation} -import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation} +import org.apache.spark.sql.hive.{HiveSessionState, MetastoreRelation} import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ @@ -45,8 +45,7 @@ private[hive] case class AnalyzeTable(tableName: String) extends RunnableCommand { override def run(sqlContext: SQLContext): Seq[Row] = { - val sessionState = sqlContext.sessionState - val hiveContext = sqlContext.asInstanceOf[HiveContext] + val sessionState = sqlContext.sessionState.asInstanceOf[HiveSessionState] val tableIdent = sessionState.sqlParser.parseTableIdentifier(tableName) val relation = EliminateSubqueryAliases(sessionState.catalog.lookupRelation(tableIdent)) @@ -60,7 +59,7 @@ case class AnalyzeTable(tableName: String) extends RunnableCommand { // Can we use fs.getContentSummary in future? // Seems fs.getContentSummary returns wrong table size on Jenkins. So we use // countFileSize to count the table size. - val stagingDir = hiveContext.metadataHive.getConf( + val stagingDir = sessionState.metadataHive.getConf( HiveConf.ConfVars.STAGINGDIR.varname, HiveConf.ConfVars.STAGINGDIR.defaultStrVal) @@ -106,7 +105,7 @@ case class AnalyzeTable(tableName: String) extends RunnableCommand { .map(_.toLong) .getOrElse(0L) val newTotalSize = - getFileSizeForTable(hiveContext.sessionState.hiveconf, relation.hiveQlTable) + getFileSizeForTable(sessionState.hiveconf, relation.hiveQlTable) // Update the Hive metastore if the total size of the table is different than the size // recorded in the Hive metastore. // This logic is based on org.apache.hadoop.hive.ql.exec.StatsTask.aggregateStats(). @@ -144,9 +143,8 @@ private[hive] case class AddFile(path: String) extends RunnableCommand { override def run(sqlContext: SQLContext): Seq[Row] = { - val hiveContext = sqlContext.asInstanceOf[HiveContext] - hiveContext.runSqlHive(s"ADD FILE $path") - hiveContext.sparkContext.addFile(path) + sqlContext.sessionState.runNativeSql(s"ADD FILE $path") + sqlContext.sparkContext.addFile(path) Seq.empty[Row] } } @@ -176,9 +174,9 @@ case class CreateMetastoreDataSource( } val tableName = tableIdent.unquotedString - val hiveContext = sqlContext.asInstanceOf[HiveContext] + val sessionState = sqlContext.sessionState.asInstanceOf[HiveSessionState] - if (hiveContext.sessionState.catalog.tableExists(tableIdent)) { + if (sessionState.catalog.tableExists(tableIdent)) { if (allowExisting) { return Seq.empty[Row] } else { @@ -190,8 +188,7 @@ case class CreateMetastoreDataSource( val optionsWithPath = if (!options.contains("path") && managedIfNoPath) { isExternal = false - options + ("path" -> - hiveContext.sessionState.catalog.hiveDefaultTableFilePath(tableIdent)) + options + ("path" -> sessionState.catalog.hiveDefaultTableFilePath(tableIdent)) } else { options } @@ -204,7 +201,7 @@ case class CreateMetastoreDataSource( bucketSpec = None, options = optionsWithPath).resolveRelation() - hiveContext.sessionState.catalog.createDataSourceTable( + sessionState.catalog.createDataSourceTable( tableIdent, userSpecifiedSchema, Array.empty[String], @@ -243,14 +240,13 @@ case class CreateMetastoreDataSourceAsSelect( } val tableName = tableIdent.unquotedString - val hiveContext = sqlContext.asInstanceOf[HiveContext] + val sessionState = sqlContext.sessionState.asInstanceOf[HiveSessionState] var createMetastoreTable = false var isExternal = true val optionsWithPath = if (!options.contains("path")) { isExternal = false - options + ("path" -> - hiveContext.sessionState.catalog.hiveDefaultTableFilePath(tableIdent)) + options + ("path" -> sessionState.catalog.hiveDefaultTableFilePath(tableIdent)) } else { options } @@ -281,14 +277,14 @@ case class CreateMetastoreDataSourceAsSelect( // inserting into (i.e. using the same compression). EliminateSubqueryAliases( - sqlContext.sessionState.catalog.lookupRelation(tableIdent)) match { + sessionState.catalog.lookupRelation(tableIdent)) match { case l @ LogicalRelation(_: InsertableRelation | _: HadoopFsRelation, _, _) => existingSchema = Some(l.schema) case o => throw new AnalysisException(s"Saving data in ${o.toString} is not supported.") } case SaveMode.Overwrite => - hiveContext.sql(s"DROP TABLE IF EXISTS $tableName") + sqlContext.sql(s"DROP TABLE IF EXISTS $tableName") // Need to create the table again. createMetastoreTable = true } @@ -297,7 +293,7 @@ case class CreateMetastoreDataSourceAsSelect( createMetastoreTable = true } - val data = Dataset.ofRows(hiveContext, query) + val data = Dataset.ofRows(sqlContext, query) val df = existingSchema match { // If we are inserting into an existing table, just use the existing schema. case Some(s) => data.selectExpr(s.fieldNames: _*) @@ -318,7 +314,7 @@ case class CreateMetastoreDataSourceAsSelect( // We will use the schema of resolved.relation as the schema of the table (instead of // the schema of df). It is important since the nullability may be changed by the relation // provider (for example, see org.apache.spark.sql.parquet.DefaultSource). - hiveContext.sessionState.catalog.createDataSourceTable( + sessionState.catalog.createDataSourceTable( tableIdent, Some(result.schema), partitionColumns, @@ -329,7 +325,7 @@ case class CreateMetastoreDataSourceAsSelect( } // Refresh the cache of the table in the catalog. - hiveContext.sessionState.catalog.refreshTable(tableIdent) + sessionState.catalog.refreshTable(tableIdent) Seq.empty[Row] } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 2767528395d9d..e629099086899 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -32,16 +32,16 @@ import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.sql.SQLContext +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.CATALOG_IMPLEMENTATION +import org.apache.spark.sql.{SparkSession, SQLContext} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions.ExpressionInfo import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.CacheManager import org.apache.spark.sql.execution.command.CacheTableCommand -import org.apache.spark.sql.execution.ui.SQLListener import org.apache.spark.sql.hive._ -import org.apache.spark.sql.hive.client.{HiveClient, HiveClientImpl} +import org.apache.spark.sql.hive.client.HiveClient import org.apache.spark.sql.hive.execution.HiveNativeCommand import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.{ShutdownHookManager, Utils} @@ -71,42 +71,80 @@ object TestHive * hive metastore seems to lead to weird non-deterministic failures. Therefore, the execution of * test cases that rely on TestHive must be serialized. */ -class TestHiveContext private[hive]( - testHiveSharedState: TestHiveSharedState, - val warehousePath: File, - val scratchDirPath: File, - metastoreTemporaryConf: Map[String, String], - isRootContext: Boolean) - extends HiveContext(testHiveSharedState, isRootContext) { self => +class TestHiveContext(@transient val sparkSession: TestHiveSparkSession, isRootContext: Boolean) + extends HiveContext(sparkSession, isRootContext) { - private def this( - sc: SparkContext, - warehousePath: File, - scratchDirPath: File, - metastoreTemporaryConf: Map[String, String]) { - this( - new TestHiveSharedState(sc, warehousePath, scratchDirPath, metastoreTemporaryConf), - warehousePath, - scratchDirPath, - metastoreTemporaryConf, - true) + def this(sc: SparkContext) { + this(new TestHiveSparkSession(HiveContext.withHiveExternalCatalog(sc)), true) + } + + override def newSession(): TestHiveContext = { + new TestHiveContext(sparkSession.newSession(), false) + } + + override def sharedState: TestHiveSharedState = sparkSession.sharedState + + override def sessionState: TestHiveSessionState = sparkSession.sessionState + + def setCacheTables(c: Boolean): Unit = { + sparkSession.setCacheTables(c) + } + + def getHiveFile(path: String): File = { + sparkSession.getHiveFile(path) + } + + def loadTestTable(name: String): Unit = { + sparkSession.loadTestTable(name) } + def reset(): Unit = { + sparkSession.reset() + } + +} + + +private[hive] class TestHiveSparkSession( + sc: SparkContext, + val warehousePath: File, + scratchDirPath: File, + metastoreTemporaryConf: Map[String, String], + existingSharedState: Option[TestHiveSharedState]) + extends SparkSession(sc) with Logging { self => + def this(sc: SparkContext) { this( sc, Utils.createTempDir(namePrefix = "warehouse"), TestHiveContext.makeScratchDir(), - HiveContext.newTemporaryConfiguration(useInMemoryDerby = false)) + HiveContext.newTemporaryConfiguration(useInMemoryDerby = false), + None) } - override def newSession(): HiveContext = { - new TestHiveContext( - testHiveSharedState, - warehousePath, - scratchDirPath, - metastoreTemporaryConf, - isRootContext = false) + assume(sc.conf.get(CATALOG_IMPLEMENTATION) == "hive") + + // TODO: Let's remove TestHiveSharedState and TestHiveSessionState. Otherwise, + // we are not really testing the reflection logic based on the setting of + // CATALOG_IMPLEMENTATION. + @transient + override lazy val sharedState: TestHiveSharedState = { + existingSharedState.getOrElse( + new TestHiveSharedState(sc, warehousePath, scratchDirPath, metastoreTemporaryConf)) + } + + @transient + override lazy val sessionState: TestHiveSessionState = new TestHiveSessionState(self) + + override def newSession(): TestHiveSparkSession = { + new TestHiveSparkSession( + sc, warehousePath, scratchDirPath, metastoreTemporaryConf, Some(sharedState)) + } + + private var cacheTables: Boolean = false + + def setCacheTables(c: Boolean): Unit = { + cacheTables = c } // By clearing the port we force Spark to pick a new one. This allows us to rerun tests @@ -118,9 +156,10 @@ class TestHiveContext private[hive]( // A snapshot of the entries in the starting SQLConf // We save this because tests can mutate this singleton object if they want + // This snapshot is saved when we create this TestHiveSparkSession. val initialSQLConf: SQLConf = { val snapshot = new SQLConf - conf.getAllConfs.foreach { case (k, v) => snapshot.setConfString(k, v) } + sessionState.conf.getAllConfs.foreach { case (k, v) => snapshot.setConfString(k, v) } snapshot } @@ -131,42 +170,10 @@ class TestHiveContext private[hive]( /** The location of the compiled hive distribution */ lazy val hiveHome = envVarToFile("HIVE_HOME") + /** The location of the hive source code. */ lazy val hiveDevHome = envVarToFile("HIVE_DEV_HOME") - // Override so we can intercept relative paths and rewrite them to point at hive. - override def runSqlHive(sql: String): Seq[String] = - super.runSqlHive(rewritePaths(substitutor.substitute(sessionState.hiveconf, sql))) - - override def executePlan(plan: LogicalPlan): this.QueryExecution = - new this.QueryExecution(plan) - - @transient - protected[sql] override lazy val sessionState = new HiveSessionState(this) { - override lazy val conf: SQLConf = { - new SQLConf { - clear() - override def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE, false) - override def clear(): Unit = { - super.clear() - TestHiveContext.overrideConfs.map { - case (key, value) => setConfString(key, value) - } - } - } - } - - override lazy val functionRegistry = { - // We use TestHiveFunctionRegistry at here to track functions that have been explicitly - // unregistered (through TestHiveFunctionRegistry.unregisterFunction method). - val fr = new TestHiveFunctionRegistry - org.apache.spark.sql.catalyst.analysis.FunctionRegistry.expressions.foreach { - case (name, (info, builder)) => fr.registerFunction(name, info, builder) - } - fr - } - } - /** * Returns the value of specified environmental variable as a [[java.io.File]] after checking * to ensure it exists @@ -179,7 +186,7 @@ class TestHiveContext private[hive]( * Replaces relative paths to the parent directory "../" with hiveDevHome since this is how the * hive test cases assume the system is set up. */ - private def rewritePaths(cmd: String): String = + private[hive] def rewritePaths(cmd: String): String = if (cmd.toUpperCase contains "LOAD DATA") { val testDataLocation = hiveDevHome.map(_.getCanonicalPath).getOrElse(inRepoTests.getCanonicalPath) @@ -210,36 +217,11 @@ class TestHiveContext private[hive]( val describedTable = "DESCRIBE (\\w+)".r - /** - * Override QueryExecution with special debug workflow. - */ - class QueryExecution(logicalPlan: LogicalPlan) - extends super.QueryExecution(logicalPlan) { - def this(sql: String) = this(parseSql(sql)) - override lazy val analyzed = { - val describedTables = logical match { - case HiveNativeCommand(describedTable(tbl)) => tbl :: Nil - case CacheTableCommand(tbl, _, _) => tbl :: Nil - case _ => Nil - } - - // Make sure any test tables referenced are loaded. - val referencedTables = - describedTables ++ - logical.collect { case UnresolvedRelation(tableIdent, _) => tableIdent.table } - val referencedTestTables = referencedTables.filter(testTables.contains) - logDebug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") - referencedTestTables.foreach(loadTestTable) - // Proceed with analysis. - sessionState.analyzer.execute(logical) - } - } - case class TestTable(name: String, commands: (() => Unit)*) protected[hive] implicit class SqlCmd(sql: String) { def cmd: () => Unit = { - () => new QueryExecution(sql).stringResult(): Unit + () => new TestHiveQueryExecution(sql).stringResult(): Unit } } @@ -266,19 +248,20 @@ class TestHiveContext private[hive]( "CREATE TABLE src1 (key INT, value STRING)".cmd, s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv3.txt")}' INTO TABLE src1".cmd), TestTable("srcpart", () => { - runSqlHive( + sessionState.runNativeSql( "CREATE TABLE srcpart (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING)") for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq("11", "12")) { - runSqlHive( + sessionState.runNativeSql( s"""LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' |OVERWRITE INTO TABLE srcpart PARTITION (ds='$ds',hr='$hr') """.stripMargin) } }), TestTable("srcpart1", () => { - runSqlHive("CREATE TABLE srcpart1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr INT)") + sessionState.runNativeSql( + "CREATE TABLE srcpart1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr INT)") for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- 11 to 12) { - runSqlHive( + sessionState.runNativeSql( s"""LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' |OVERWRITE INTO TABLE srcpart1 PARTITION (ds='$ds',hr='$hr') """.stripMargin) @@ -289,7 +272,7 @@ class TestHiveContext private[hive]( import org.apache.hadoop.mapred.{SequenceFileInputFormat, SequenceFileOutputFormat} import org.apache.thrift.protocol.TBinaryProtocol - runSqlHive( + sessionState.runNativeSql( s""" |CREATE TABLE src_thrift(fake INT) |ROW FORMAT SERDE '${classOf[ThriftDeserializer].getName}' @@ -302,7 +285,7 @@ class TestHiveContext private[hive]( |OUTPUTFORMAT '${classOf[SequenceFileOutputFormat[_, _]].getName}' """.stripMargin) - runSqlHive( + sessionState.runNativeSql( s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/complex.seq")}' INTO TABLE src_thrift") }), TestTable("serdeins", @@ -415,7 +398,6 @@ class TestHiveContext private[hive]( private val loadedTables = new collection.mutable.HashSet[String] - var cacheTables: Boolean = false def loadTestTable(name: String) { if (!(loadedTables contains name)) { // Marks the table as loaded first to prevent infinite mutually recursive table loading. @@ -426,7 +408,7 @@ class TestHiveContext private[hive]( createCmds.foreach(_()) if (cacheTables) { - cacheTable(name) + new SQLContext(self).cacheTable(name) } } } @@ -451,11 +433,12 @@ class TestHiveContext private[hive]( } } - cacheManager.clearCache() + sharedState.cacheManager.clearCache() loadedTables.clear() sessionState.catalog.clearTempTables() sessionState.catalog.invalidateCache() - metadataHive.reset() + + sessionState.metadataHive.reset() FunctionRegistry.getFunctionNames.asScala.filterNot(originalUDFs.contains(_)). foreach { udfName => FunctionRegistry.unregisterTemporaryUDF(udfName) } @@ -464,21 +447,21 @@ class TestHiveContext private[hive]( sessionState.hiveconf.set("fs.default.name", new File(".").toURI.toString) // It is important that we RESET first as broken hooks that might have been set could break // other sql exec here. - executionHive.runSqlHive("RESET") - metadataHive.runSqlHive("RESET") + sessionState.executionHive.runSqlHive("RESET") + sessionState.metadataHive.runSqlHive("RESET") // For some reason, RESET does not reset the following variables... // https://issues.apache.org/jira/browse/HIVE-9004 - runSqlHive("set hive.table.parameters.default=") - runSqlHive("set datanucleus.cache.collections=true") - runSqlHive("set datanucleus.cache.collections.lazy=true") + sessionState.runNativeSql("set hive.table.parameters.default=") + sessionState.runNativeSql("set datanucleus.cache.collections=true") + sessionState.runNativeSql("set datanucleus.cache.collections.lazy=true") // Lots of tests fail if we do not change the partition whitelist from the default. - runSqlHive("set hive.metastore.partition.name.whitelist.pattern=.*") + sessionState.runNativeSql("set hive.metastore.partition.name.whitelist.pattern=.*") // In case a test changed any of these values, restore all the original ones here. TestHiveContext.hiveClientConfigurations( sessionState.hiveconf, warehousePath, scratchDirPath, metastoreTemporaryConf) - .foreach { case (k, v) => metadataHive.runSqlHive(s"SET $k=$v") } - defaultOverrides() + .foreach { case (k, v) => sessionState.metadataHive.runSqlHive(s"SET $k=$v") } + sessionState.setDefaultOverrideConfs() sessionState.catalog.setCurrentDatabase("default") } catch { @@ -489,6 +472,40 @@ class TestHiveContext private[hive]( } + +private[hive] class TestHiveQueryExecution( + sparkSession: TestHiveSparkSession, + logicalPlan: LogicalPlan) + extends HiveQueryExecution(new SQLContext(sparkSession), logicalPlan) with Logging { + + def this(sparkSession: TestHiveSparkSession, sql: String) { + this(sparkSession, sparkSession.sessionState.sqlParser.parsePlan(sql)) + } + + def this(sql: String) { + this(TestHive.sparkSession, sql) + } + + override lazy val analyzed: LogicalPlan = { + val describedTables = logical match { + case HiveNativeCommand(sparkSession.describedTable(tbl)) => tbl :: Nil + case CacheTableCommand(tbl, _, _) => tbl :: Nil + case _ => Nil + } + + // Make sure any test tables referenced are loaded. + val referencedTables = + describedTables ++ + logical.collect { case UnresolvedRelation(tableIdent, _) => tableIdent.table } + val referencedTestTables = referencedTables.filter(sparkSession.testTables.contains) + logDebug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") + referencedTestTables.foreach(sparkSession.loadTestTable) + // Proceed with analysis. + sparkSession.sessionState.analyzer.execute(logical) + } +} + + private[hive] class TestHiveFunctionRegistry extends SimpleFunctionRegistry { private val removedFunctions = @@ -517,7 +534,43 @@ private[hive] class TestHiveSharedState( TestHiveContext.newClientForMetadata( sc.conf, sc.hadoopConfiguration, warehousePath, scratchDirPath, metastoreTemporaryConf) } +} + + +private[hive] class TestHiveSessionState(sparkSession: TestHiveSparkSession) + extends HiveSessionState(new SQLContext(sparkSession)) { + override lazy val conf: SQLConf = { + new SQLConf { + clear() + override def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE, false) + override def clear(): Unit = { + super.clear() + TestHiveContext.overrideConfs.map { + case (key, value) => setConfString(key, value) + } + } + } + } + + override lazy val functionRegistry: TestHiveFunctionRegistry = { + // We use TestHiveFunctionRegistry at here to track functions that have been explicitly + // unregistered (through TestHiveFunctionRegistry.unregisterFunction method). + val fr = new TestHiveFunctionRegistry + org.apache.spark.sql.catalyst.analysis.FunctionRegistry.expressions.foreach { + case (name, (info, builder)) => fr.registerFunction(name, info, builder) + } + fr + } + + override def executePlan(plan: LogicalPlan): TestHiveQueryExecution = { + new TestHiveQueryExecution(sparkSession, plan) + } + + // Override so we can intercept relative paths and rewrite them to point at hive. + override def runNativeSql(sql: String): Seq[String] = { + super.runNativeSql(sparkSession.rewritePaths(substitutor.substitute(hiveconf, sql))) + } } @@ -552,7 +605,7 @@ private[hive] object TestHiveContext { /** * Configurations needed to create a [[HiveClient]]. */ - private def hiveClientConfigurations( + def hiveClientConfigurations( hiveconf: HiveConf, warehousePath: File, scratchDirPath: File, @@ -564,7 +617,7 @@ private[hive] object TestHiveContext { ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY.varname -> "1") } - private def makeScratchDir(): File = { + def makeScratchDir(): File = { val scratchDir = Utils.createTempDir(namePrefix = "scratch") scratchDir.delete() scratchDir diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala index b9e7a36b41a1a..61910b8e6b51d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala @@ -23,7 +23,6 @@ import org.scalatest.BeforeAndAfterEach import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.catalyst.util.quietly -import org.apache.spark.sql.hive.execution.HiveSqlParser import org.apache.spark.sql.hive.test.TestHiveSingleton class ErrorPositionSuite extends QueryTest with TestHiveSingleton with BeforeAndAfterEach { @@ -131,7 +130,7 @@ class ErrorPositionSuite extends QueryTest with TestHiveSingleton with BeforeAnd * @param token a unique token in the string that should be indicated by the exception */ def positionTest(name: String, query: String, token: String): Unit = { - def ast = hiveContext.sessionState.sqlParser.parsePlan(query) + def ast = hiveContext.parseSql(query) def parseTree = Try(quietly(ast.treeString)).getOrElse("") test(name) { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextSuite.scala index b644a50613337..b2c0f7e0e57b4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextSuite.scala @@ -28,9 +28,12 @@ class HiveContextSuite extends SparkFunSuite { val sc = TestHive.sparkContext require(sc.conf.get("spark.sql.hive.metastore.barrierPrefixes") == "org.apache.spark.sql.hive.execution.PairSerDe") - assert(TestHive.initialSQLConf.getConfString("spark.sql.hive.metastore.barrierPrefixes") == + assert(TestHive.sparkSession.initialSQLConf.getConfString( + "spark.sql.hive.metastore.barrierPrefixes") == "org.apache.spark.sql.hive.execution.PairSerDe") - assert(TestHive.metadataHive.getConf("spark.sql.hive.metastore.barrierPrefixes", "") == + // This setting should be also set in the hiveconf of the current session. + assert(TestHive.sessionState.hiveconf.get( + "spark.sql.hive.metastore.barrierPrefixes", "") == "org.apache.spark.sql.hive.execution.PairSerDe") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index 8648834f0d881..2a201c195f167 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -96,7 +96,7 @@ class DataSourceWithHiveMetastoreCatalogSuite assert(columns.map(_.dataType) === Seq("decimal(10,3)", "string")) checkAnswer(table("t"), testDF) - assert(runSqlHive("SELECT * FROM t") === Seq("1.1\t1", "2.1\t2")) + assert(sessionState.runNativeSql("SELECT * FROM t") === Seq("1.1\t1", "2.1\t2")) } } @@ -129,7 +129,7 @@ class DataSourceWithHiveMetastoreCatalogSuite assert(columns.map(_.dataType) === Seq("decimal(10,3)", "string")) checkAnswer(table("t"), testDF) - assert(runSqlHive("SELECT * FROM t") === Seq("1.1\t1", "2.1\t2")) + assert(sessionState.runNativeSql("SELECT * FROM t") === Seq("1.1\t1", "2.1\t2")) } } } @@ -159,7 +159,7 @@ class DataSourceWithHiveMetastoreCatalogSuite assert(columns.map(_.dataType) === Seq("int", "string")) checkAnswer(table("t"), Row(1, "val_1")) - assert(runSqlHive("SELECT * FROM t") === Seq("1\tval_1")) + assert(sessionState.runNativeSql("SELECT * FROM t") === Seq("1\tval_1")) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index d0e687051914e..bbe135b2d6980 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -253,13 +253,13 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv sql("SELECT `c_!@(3)` FROM expectedJsonTable").collect().toSeq) // Discard the cached relation. - invalidateTable("jsonTable") + sessionState.invalidateTable("jsonTable") checkAnswer( sql("SELECT * FROM jsonTable"), sql("SELECT `c_!@(3)` FROM expectedJsonTable").collect().toSeq) - invalidateTable("jsonTable") + sessionState.invalidateTable("jsonTable") val expectedSchema = StructType(StructField("c_!@(3)", IntegerType, true) :: Nil) assert(expectedSchema === table("jsonTable").schema) @@ -347,7 +347,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv """.stripMargin) // Discard the cached relation. - invalidateTable("ctasJsonTable") + sessionState.invalidateTable("ctasJsonTable") // Schema should not be changed. assert(table("ctasJsonTable").schema === table("jsonTable").schema) @@ -422,7 +422,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv sql("SELECT * FROM savedJsonTable tmp where tmp.a > 5"), (6 to 10).map(i => Row(i, s"str$i"))) - invalidateTable("savedJsonTable") + sessionState.invalidateTable("savedJsonTable") checkAnswer( sql("SELECT * FROM savedJsonTable where savedJsonTable.a < 5"), @@ -620,7 +620,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv .mode(SaveMode.Append) .saveAsTable("arrayInParquet") - refreshTable("arrayInParquet") + sessionState.refreshTable("arrayInParquet") checkAnswer( sql("SELECT a FROM arrayInParquet"), @@ -679,7 +679,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv .mode(SaveMode.Append) .saveAsTable("mapInParquet") - refreshTable("mapInParquet") + sessionState.refreshTable("mapInParquet") checkAnswer( sql("SELECT a FROM mapInParquet"), @@ -707,7 +707,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv options = Map("path" -> tempDir.getCanonicalPath), isExternal = false) - invalidateTable("wide_schema") + sessionState.invalidateTable("wide_schema") val actualSchema = table("wide_schema").schema assert(schema === actualSchema) @@ -737,9 +737,9 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv "spark.sql.sources.schema" -> schema.json, "EXTERNAL" -> "FALSE")) - hiveCatalog.createTable("default", hiveTable, ignoreIfExists = false) + sharedState.externalCatalog.createTable("default", hiveTable, ignoreIfExists = false) - invalidateTable(tableName) + sessionState.invalidateTable(tableName) val actualSchema = table(tableName).schema assert(schema === actualSchema) } @@ -751,8 +751,8 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv withTable(tableName) { df.write.format("parquet").partitionBy("d", "b").saveAsTable(tableName) - invalidateTable(tableName) - val metastoreTable = hiveCatalog.getTable("default", tableName) + sessionState.invalidateTable(tableName) + val metastoreTable = sharedState.externalCatalog.getTable("default", tableName) val expectedPartitionColumns = StructType(df.schema("d") :: df.schema("b") :: Nil) val numPartCols = metastoreTable.properties("spark.sql.sources.schema.numPartCols").toInt @@ -786,8 +786,8 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv .bucketBy(8, "d", "b") .sortBy("c") .saveAsTable(tableName) - invalidateTable(tableName) - val metastoreTable = hiveCatalog.getTable("default", tableName) + sessionState.invalidateTable(tableName) + val metastoreTable = sharedState.externalCatalog.getTable("default", tableName) val expectedBucketByColumns = StructType(df.schema("d") :: df.schema("b") :: Nil) val expectedSortByColumns = StructType(df.schema("c") :: Nil) @@ -917,7 +917,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv // As a proxy for verifying that the table was stored in Hive compatible format, // we verify that each column of the table is of native type StringType. - assert(hiveCatalog.getTable("default", "not_skip_hive_metadata").schema + assert(sharedState.externalCatalog.getTable("default", "not_skip_hive_metadata").schema .forall(column => HiveMetastoreTypes.toDataType(column.dataType) == StringType)) sessionState.catalog.createDataSourceTable( @@ -931,9 +931,8 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv // As a proxy for verifying that the table was stored in SparkSQL format, // we verify that the table has a column type as array of StringType. - assert(hiveCatalog.getTable("default", "skip_hive_metadata").schema.forall { c => - HiveMetastoreTypes.toDataType(c.dataType) == ArrayType(StringType) - }) + assert(sharedState.externalCatalog.getTable("default", "skip_hive_metadata") + .schema.forall { c => HiveMetastoreTypes.toDataType(c.dataType) == ArrayType(StringType) }) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala index 3c003506efcb1..850cb1eda5809 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala @@ -25,8 +25,9 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle private lazy val df = sqlContext.range(10).coalesce(1).toDF() private def checkTablePath(dbName: String, tableName: String): Unit = { - val metastoreTable = hiveContext.hiveCatalog.getTable(dbName, tableName) - val expectedPath = hiveContext.hiveCatalog.getDatabase(dbName).locationUri + "/" + tableName + val metastoreTable = hiveContext.sharedState.externalCatalog.getTable(dbName, tableName) + val expectedPath = + hiveContext.sharedState.externalCatalog.getDatabase(dbName).locationUri + "/" + tableName assert(metastoreTable.storage.serdeProperties("path") === expectedPath) } @@ -216,7 +217,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle df.write.parquet(s"$path/p=2") sql("ALTER TABLE t ADD PARTITION (p=2)") - hiveContext.refreshTable("t") + hiveContext.sessionState.refreshTable("t") checkAnswer( sqlContext.table("t"), df.withColumn("p", lit(1)).union(df.withColumn("p", lit(2)))) @@ -248,7 +249,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle df.write.parquet(s"$path/p=2") sql(s"ALTER TABLE $db.t ADD PARTITION (p=2)") - hiveContext.refreshTable(s"$db.t") + hiveContext.sessionState.refreshTable(s"$db.t") checkAnswer( sqlContext.table(s"$db.t"), df.withColumn("p", lit(1)).union(df.withColumn("p", lit(2)))) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index d14c72b34bc9a..adc7af32ca03f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -31,7 +31,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { test("parse analyze commands") { def assertAnalyzeCommand(analyzeCommand: String, c: Class[_]) { - val parsed = hiveContext.sessionState.sqlParser.parsePlan(analyzeCommand) + val parsed = hiveContext.parseSql(analyzeCommand) val operators = parsed.collect { case a: AnalyzeTable => a case o => o @@ -116,7 +116,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton { // Try to analyze a temp table sql("""SELECT * FROM src""").registerTempTable("tempTable") intercept[UnsupportedOperationException] { - hiveContext.analyze("tempTable") + hiveContext.sessionState.analyze("tempTable") } hiveContext.sessionState.catalog.dropTable( TableIdentifier("tempTable"), ignoreIfNotExists = true) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala index a3f5921a0cb23..c58a66418991b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala @@ -19,15 +19,15 @@ package org.apache.spark.sql.hive.execution import java.io.File -import org.apache.spark.sql.hive.test.TestHive._ /** * A set of test cases based on the big-data-benchmark. * https://amplab.cs.berkeley.edu/benchmark/ */ class BigDataBenchmarkSuite extends HiveComparisonTest { - val testDataDirectory = new File("target" + File.separator + "big-data-benchmark-testdata") + import org.apache.spark.sql.hive.test.TestHive.sparkSession._ + val testDataDirectory = new File("target" + File.separator + "big-data-benchmark-testdata") val userVisitPath = new File(testDataDirectory, "uservisits").getCanonicalPath val testTables = Seq( TestTable( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index e67fcbedc3364..bd46cb922e1ba 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -30,8 +30,9 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.command.{ExplainCommand, SetCommand} import org.apache.spark.sql.execution.datasources.DescribeCommand -import org.apache.spark.sql.hive.{InsertIntoHiveTable => LogicalInsertIntoHiveTable, SQLBuilder} -import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.{InsertIntoHiveTable => LogicalInsertIntoHiveTable} +import org.apache.spark.sql.hive.SQLBuilder +import org.apache.spark.sql.hive.test.{TestHive, TestHiveQueryExecution} /** * Allows the creations of tests that execute the same query against both hive @@ -141,7 +142,7 @@ abstract class HiveComparisonTest } protected def prepareAnswer( - hiveQuery: TestHive.type#QueryExecution, + hiveQuery: TestHiveQueryExecution, answer: Seq[String]): Seq[String] = { def isSorted(plan: LogicalPlan): Boolean = plan match { @@ -332,7 +333,7 @@ abstract class HiveComparisonTest hiveCachedResults } else { - val hiveQueries = queryList.map(new TestHive.QueryExecution(_)) + val hiveQueries = queryList.map(new TestHiveQueryExecution(_)) // Make sure we can at least parse everything before attempting hive execution. // Note this must only look at the logical plan as we might not be able to analyze if // other DDL has not been executed yet. @@ -352,7 +353,7 @@ abstract class HiveComparisonTest case _: ExplainCommand => // No need to execute EXPLAIN queries as we don't check the output. Nil - case _ => TestHive.runSqlHive(queryString) + case _ => TestHive.sessionState.runNativeSql(queryString) } // We need to add a new line to non-empty answers so we can differentiate Seq() @@ -382,10 +383,10 @@ abstract class HiveComparisonTest // Run w/ catalyst val catalystResults = queryList.zip(hiveResults).map { case (queryString, hive) => - var query: TestHive.QueryExecution = null + var query: TestHiveQueryExecution = null try { query = { - val originalQuery = new TestHive.QueryExecution(queryString) + val originalQuery = new TestHiveQueryExecution(queryString) val containsCommands = originalQuery.analyzed.collectFirst { case _: Command => () case _: LogicalInsertIntoHiveTable => () @@ -409,7 +410,7 @@ abstract class HiveComparisonTest } try { - val queryExecution = new TestHive.QueryExecution(convertedSQL) + val queryExecution = new TestHiveQueryExecution(convertedSQL) // Trigger the analysis of this converted SQL query. queryExecution.analyzed queryExecution @@ -472,12 +473,12 @@ abstract class HiveComparisonTest // If this query is reading other tables that were created during this test run // also print out the query plans and results for those. val computedTablesMessages: String = try { - val tablesRead = new TestHive.QueryExecution(query).executedPlan.collect { + val tablesRead = new TestHiveQueryExecution(query).executedPlan.collect { case ts: HiveTableScan => ts.relation.tableName }.toSet TestHive.reset() - val executions = queryList.map(new TestHive.QueryExecution(_)) + val executions = queryList.map(new TestHiveQueryExecution(_)) executions.foreach(_.toRdd) val tablesGenerated = queryList.zip(executions).flatMap { // We should take executedPlan instead of sparkPlan, because in following codes we @@ -562,8 +563,8 @@ abstract class HiveComparisonTest // okay by running a simple query. If this fails then we halt testing since // something must have gone seriously wrong. try { - new TestHive.QueryExecution("SELECT key FROM src").stringResult() - TestHive.runSqlHive("SELECT key FROM src") + new TestHiveQueryExecution("SELECT key FROM src").stringResult() + TestHive.sessionState.runNativeSql("SELECT key FROM src") } catch { case e: Exception => logError(s"FATAL ERROR: Canary query threw $e This implies that the " + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 2e7a1d921b75c..93d63f2241324 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -49,7 +49,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { override def beforeAll() { super.beforeAll() - TestHive.cacheTables = true + TestHive.setCacheTables(true) // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) // Add Locale setting @@ -58,7 +58,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { override def afterAll() { try { - TestHive.cacheTables = false + TestHive.setCacheTables(false) TimeZone.setDefault(originalTimeZone) Locale.setDefault(originalLocale) sql("DROP TEMPORARY FUNCTION IF EXISTS udtf_count2") @@ -1009,7 +1009,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { .mkString("/") // Loads partition data to a temporary table to verify contents - val path = s"$warehousePath/dynamic_part_table/$partFolder/part-00000" + val path = s"${sparkSession.warehousePath}/dynamic_part_table/$partFolder/part-00000" sql("DROP TABLE IF EXISTS dp_verify") sql("CREATE TABLE dp_verify(intcol INT)") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala index 5586a793618bd..b8af0b39c8392 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala @@ -28,8 +28,8 @@ class HiveSerDeSuite extends HiveComparisonTest with BeforeAndAfterAll { override def beforeAll(): Unit = { import TestHive._ import org.apache.hadoop.hive.serde2.RegexSerDe - super.beforeAll() - TestHive.cacheTables = false + super.beforeAll() + TestHive.setCacheTables(false) sql(s"""CREATE TABLE IF NOT EXISTS sales (key STRING, value INT) |ROW FORMAT SERDE '${classOf[RegexSerDe].getCanonicalName}' |WITH SERDEPROPERTIES ("input.regex" = "([^ ]*)\t([^ ]*)") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 97cb9d972081c..79ac53c8630f9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -21,18 +21,22 @@ import scala.collection.JavaConverters._ import org.scalatest.BeforeAndAfter -import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.{TestHive, TestHiveQueryExecution} /** * A set of test cases that validate partition and column pruning. */ class PruningSuite extends HiveComparisonTest with BeforeAndAfter { - TestHive.cacheTables = false - // Column/partition pruning is not implemented for `InMemoryColumnarTableScan` yet, need to reset - // the environment to ensure all referenced tables in this suites are not cached in-memory. - // Refer to https://issues.apache.org/jira/browse/SPARK-2283 for details. - TestHive.reset() + override def beforeAll(): Unit = { + super.beforeAll() + TestHive.setCacheTables(false) + // Column/partition pruning is not implemented for `InMemoryColumnarTableScan` yet, + // need to reset the environment to ensure all referenced tables in this suites are + // not cached in-memory. Refer to https://issues.apache.org/jira/browse/SPARK-2283 + // for details. + TestHive.reset() + } // Column pruning tests @@ -144,7 +148,7 @@ class PruningSuite extends HiveComparisonTest with BeforeAndAfter { expectedScannedColumns: Seq[String], expectedPartValues: Seq[Seq[String]]): Unit = { test(s"$testCaseName - pruning test") { - val plan = new TestHive.QueryExecution(sql).sparkPlan + val plan = new TestHiveQueryExecution(sql).sparkPlan val actualOutputColumns = plan.output.map(_.name) val (actualScannedColumns, actualPartValues) = plan.collect { case p @ HiveTableScan(columns, relation, _) => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 1098e74cab684..6b71e59b7359c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -349,7 +349,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } - val originalConf = convertCTAS + val originalConf = sessionState.convertCTAS setConf(HiveContext.CONVERT_CTAS, true) @@ -731,7 +731,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { // generates an invalid query plan. val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i + 1}]}""")) read.json(rdd).registerTempTable("data") - val originalConf = convertCTAS + val originalConf = sessionState.convertCTAS setConf(HiveContext.CONVERT_CTAS, false) try { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala index 8f163f27c94cf..00b5c8dd41730 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala @@ -58,7 +58,7 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { output = Seq(AttributeReference("a", StringType)()), child = child, ioschema = noSerdeIOSchema - )(hiveContext), + )(hiveContext.sessionState.hiveconf), rowsDf.collect()) } @@ -72,7 +72,7 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { output = Seq(AttributeReference("a", StringType)()), child = child, ioschema = serdeIOSchema - )(hiveContext), + )(hiveContext.sessionState.hiveconf), rowsDf.collect()) } @@ -87,7 +87,7 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { output = Seq(AttributeReference("a", StringType)()), child = ExceptionInjectingOperator(child), ioschema = noSerdeIOSchema - )(hiveContext), + )(hiveContext.sessionState.hiveconf), rowsDf.collect()) } assert(e.getMessage().contains("intentional exception")) @@ -104,7 +104,7 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { output = Seq(AttributeReference("a", StringType)()), child = ExceptionInjectingOperator(child), ioschema = serdeIOSchema - )(hiveContext), + )(hiveContext.sessionState.hiveconf), rowsDf.collect()) } assert(e.getMessage().contains("intentional exception")) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index 4b2b1a160ad5c..6fa4c3334fa72 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -461,7 +461,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { checkCached(tableIdentifier) // For insert into non-partitioned table, we will do the conversion, // so the converted test_insert_parquet should be cached. - invalidateTable("test_insert_parquet") + sessionState.invalidateTable("test_insert_parquet") assert(sessionState.catalog.getCachedDataSourceTable(tableIdentifier) === null) sql( """ @@ -474,7 +474,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { sql("select * from test_insert_parquet"), sql("select a, b from jt").collect()) // Invalidate the cache. - invalidateTable("test_insert_parquet") + sessionState.invalidateTable("test_insert_parquet") assert(sessionState.catalog.getCachedDataSourceTable(tableIdentifier) === null) // Create a partitioned table. @@ -524,7 +524,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { |select b, '2015-04-02', a FROM jt """.stripMargin).collect()) - invalidateTable("test_parquet_partitioned_cache_test") + sessionState.invalidateTable("test_parquet_partitioned_cache_test") assert(sessionState.catalog.getCachedDataSourceTable(tableIdentifier) === null) dropTables("test_insert_parquet", "test_parquet_partitioned_cache_test") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index a0be55cfba94c..aa6101f7b73cf 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -349,7 +349,7 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet test("error if there exists any malformed bucket files") { withTable("bucketed_table") { df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") - val tableDir = new File(hiveContext.warehousePath, "bucketed_table") + val tableDir = new File(hiveContext.sparkSession.warehousePath, "bucketed_table") Utils.deleteRecursively(tableDir) df1.write.parquet(tableDir.getAbsolutePath) From 296c384aff5bc1c0e8d411669f8f3e082cdf8f55 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Wed, 20 Apr 2016 13:02:37 -0700 Subject: [PATCH 62/69] [MINOR][ML][PYSPARK] Fix omissive params which should use TypeConverter ## What changes were proposed in this pull request? #11663 adds type conversion functionality for parameters in Pyspark. This PR find out the omissive ```Param``` that did not pass corresponding ```TypeConverter``` argument and fix them. After this PR, all params in pyspark/ml/ used ```TypeConverter```. ## How was this patch tested? Existing tests. cc jkbradley sethah Author: Yanbo Liang Closes #12529 from yanboliang/typeConverter. --- python/pyspark/ml/regression.py | 7 ++++--- python/pyspark/ml/tuning.py | 2 +- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index da74ab5070797..8e76070e9a990 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -561,7 +561,7 @@ class TreeRegressorParams(Params): impurity = Param(Params._dummy(), "impurity", "Criterion used for information gain calculation (case-insensitive). " + "Supported options: " + - ", ".join(supportedImpurities)) + ", ".join(supportedImpurities), typeConverter=TypeConverters.toString) def __init__(self): super(TreeRegressorParams, self).__init__() @@ -1261,11 +1261,12 @@ class GeneralizedLinearRegression(JavaEstimator, HasLabelCol, HasFeaturesCol, Ha family = Param(Params._dummy(), "family", "The name of family which is a description of " + "the error distribution to be used in the model. Supported options: " + - "gaussian(default), binomial, poisson and gamma.") + "gaussian(default), binomial, poisson and gamma.", + typeConverter=TypeConverters.toString) link = Param(Params._dummy(), "link", "The name of link function which provides the " + "relationship between the linear predictor and the mean of the distribution " + "function. Supported options: identity, log, inverse, logit, probit, cloglog " + - "and sqrt.") + "and sqrt.", typeConverter=TypeConverters.toString) @keyword_only def __init__(self, labelCol="label", featuresCol="features", predictionCol="prediction", diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index ef14da488e44f..b16628bc70135 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -448,7 +448,7 @@ class TrainValidationSplit(Estimator, ValidatorParams, MLReadable, MLWritable): """ trainRatio = Param(Params._dummy(), "trainRatio", "Param for ratio between train and\ - validation data. Must be between 0 and 1.") + validation data. Must be between 0 and 1.", typeConverter=TypeConverters.toFloat) @keyword_only def __init__(self, estimator=None, estimatorParamMaps=None, evaluator=None, trainRatio=0.75, From 7bc948557bb6169cbeec335f8400af09375a62d3 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 20 Apr 2016 13:33:04 -0700 Subject: [PATCH 63/69] [SPARK-14678][SQL] Add a file sink log to support versioning and compaction ## What changes were proposed in this pull request? This PR adds a special log for FileStreamSink for two purposes: - Versioning. A future Spark version should be able to read the metadata of an old FileStreamSink. - Compaction. As reading from many small files is usually pretty slow, we should compact small metadata files into big files. FileStreamSinkLog has a new log format instead of Java serialization format. It will write one log file for each batch. The first line of the log file is the version number, and there are multiple JSON lines following. Each JSON line is a JSON format of FileLog. FileStreamSinkLog will compact log files every "spark.sql.sink.file.log.compactLen" batches into a big file. When doing a compact, it will read all history logs and merge them with the new batch. During the compaction, it will also delete the files that are deleted (marked by FileLog.action). When the reader uses allLogs to list all files, this method only returns the visible files (drops the deleted files). ## How was this patch tested? FileStreamSinkLogSuite Author: Shixiong Zhu Closes #12435 from zsxwing/sink-log. --- .../execution/streaming/FileStreamSink.scala | 26 +- .../streaming/FileStreamSinkLog.scala | 278 ++++++++++++++++++ .../execution/streaming/HDFSMetadataLog.scala | 43 ++- .../streaming/StreamFileCatalog.scala | 4 +- .../apache/spark/sql/internal/SQLConf.scala | 22 ++ .../streaming/FileStreamSinkLogSuite.scala | 270 +++++++++++++++++ 6 files changed, 616 insertions(+), 27 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala index f3c1cc5ef5ed3..4f722a514ba69 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala @@ -44,28 +44,33 @@ class FileStreamSink( private val basePath = new Path(path) private val logPath = new Path(basePath, FileStreamSink.metadataDir) - private val fileLog = new HDFSMetadataLog[Seq[String]](sqlContext, logPath.toUri.toString) + private val fileLog = new FileStreamSinkLog(sqlContext, logPath.toUri.toString) + private val fs = basePath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) override def addBatch(batchId: Long, data: DataFrame): Unit = { - if (fileLog.get(batchId).isDefined) { + if (batchId <= fileLog.getLatest().map(_._1).getOrElse(-1L)) { logInfo(s"Skipping already committed batch $batchId") } else { - val files = writeFiles(data) + val files = fs.listStatus(writeFiles(data)).map { f => + SinkFileStatus( + path = f.getPath.toUri.toString, + size = f.getLen, + isDir = f.isDirectory, + modificationTime = f.getModificationTime, + blockReplication = f.getReplication, + blockSize = f.getBlockSize, + action = FileStreamSinkLog.ADD_ACTION) + } if (fileLog.add(batchId, files)) { logInfo(s"Committed batch $batchId") } else { - logWarning(s"Race while writing batch $batchId") + throw new IllegalStateException(s"Race while writing batch $batchId") } } } /** Writes the [[DataFrame]] to a UUID-named dir, returning the list of files paths. */ - private def writeFiles(data: DataFrame): Seq[String] = { - val ctx = sqlContext - val outputDir = path - val format = fileFormat - val schema = data.schema - + private def writeFiles(data: DataFrame): Array[Path] = { val file = new Path(basePath, UUID.randomUUID().toString).toUri.toString data.write.parquet(file) sqlContext.read @@ -74,7 +79,6 @@ class FileStreamSink( .inputFiles .map(new Path(_)) .filterNot(_.getName.startsWith("_")) - .map(_.toUri.toString) } override def toString: String = s"FileSink[$path]" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala new file mode 100644 index 0000000000000..6c5449a928293 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala @@ -0,0 +1,278 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import java.io.IOException +import java.nio.charset.StandardCharsets.UTF_8 + +import org.apache.hadoop.fs.{FileStatus, Path, PathFilter} +import org.json4s.NoTypeHints +import org.json4s.jackson.Serialization +import org.json4s.jackson.Serialization.{read, write} + +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.internal.SQLConf + +/** + * The status of a file outputted by [[FileStreamSink]]. A file is visible only if it appears in + * the sink log and its action is not "delete". + * + * @param path the file path. + * @param size the file size. + * @param isDir whether this file is a directory. + * @param modificationTime the file last modification time. + * @param blockReplication the block replication. + * @param blockSize the block size. + * @param action the file action. Must be either "add" or "delete". + */ +case class SinkFileStatus( + path: String, + size: Long, + isDir: Boolean, + modificationTime: Long, + blockReplication: Int, + blockSize: Long, + action: String) { + + def toFileStatus: FileStatus = { + new FileStatus(size, isDir, blockReplication, blockSize, modificationTime, new Path(path)) + } +} + +/** + * A special log for [[FileStreamSink]]. It will write one log file for each batch. The first line + * of the log file is the version number, and there are multiple JSON lines following. Each JSON + * line is a JSON format of [[SinkFileStatus]]. + * + * As reading from many small files is usually pretty slow, [[FileStreamSinkLog]] will compact log + * files every "spark.sql.sink.file.log.compactLen" batches into a big file. When doing a + * compaction, it will read all old log files and merge them with the new batch. During the + * compaction, it will also delete the files that are deleted (marked by [[SinkFileStatus.action]]). + * When the reader uses `allFiles` to list all files, this method only returns the visible files + * (drops the deleted files). + */ +class FileStreamSinkLog(sqlContext: SQLContext, path: String) + extends HDFSMetadataLog[Seq[SinkFileStatus]](sqlContext, path) { + + import FileStreamSinkLog._ + + private implicit val formats = Serialization.formats(NoTypeHints) + + /** + * If we delete the old files after compaction at once, there is a race condition in S3: other + * processes may see the old files are deleted but still cannot see the compaction file using + * "list". The `allFiles` handles this by looking for the next compaction file directly, however, + * a live lock may happen if the compaction happens too frequently: one processing keeps deleting + * old files while another one keeps retrying. Setting a reasonable cleanup delay could avoid it. + */ + private val fileCleanupDelayMs = sqlContext.getConf(SQLConf.FILE_SINK_LOG_CLEANUP_DELAY) + + private val isDeletingExpiredLog = sqlContext.getConf(SQLConf.FILE_SINK_LOG_DELETION) + + private val compactInterval = sqlContext.getConf(SQLConf.FILE_SINK_LOG_COMPACT_INTERVAL) + require(compactInterval > 0, + s"Please set ${SQLConf.FILE_SINK_LOG_COMPACT_INTERVAL.key} (was $compactInterval) " + + "to a positive value.") + + override def batchIdToPath(batchId: Long): Path = { + if (isCompactionBatch(batchId, compactInterval)) { + new Path(metadataPath, s"$batchId$COMPACT_FILE_SUFFIX") + } else { + new Path(metadataPath, batchId.toString) + } + } + + override def pathToBatchId(path: Path): Long = { + getBatchIdFromFileName(path.getName) + } + + override def isBatchFile(path: Path): Boolean = { + try { + getBatchIdFromFileName(path.getName) + true + } catch { + case _: NumberFormatException => false + } + } + + override def serialize(logData: Seq[SinkFileStatus]): Array[Byte] = { + (VERSION +: logData.map(write(_))).mkString("\n").getBytes(UTF_8) + } + + override def deserialize(bytes: Array[Byte]): Seq[SinkFileStatus] = { + val lines = new String(bytes, UTF_8).split("\n") + if (lines.length == 0) { + throw new IllegalStateException("Incomplete log file") + } + val version = lines(0) + if (version != VERSION) { + throw new IllegalStateException(s"Unknown log version: ${version}") + } + lines.toSeq.slice(1, lines.length).map(read[SinkFileStatus](_)) + } + + override def add(batchId: Long, logs: Seq[SinkFileStatus]): Boolean = { + if (isCompactionBatch(batchId, compactInterval)) { + compact(batchId, logs) + } else { + super.add(batchId, logs) + } + } + + /** + * Returns all files except the deleted ones. + */ + def allFiles(): Array[SinkFileStatus] = { + var latestId = getLatest().map(_._1).getOrElse(-1L) + // There is a race condition when `FileStreamSink` is deleting old files and `StreamFileCatalog` + // is calling this method. This loop will retry the reading to deal with the + // race condition. + while (true) { + if (latestId >= 0) { + val startId = getAllValidBatches(latestId, compactInterval)(0) + try { + val logs = get(Some(startId), Some(latestId)).flatMap(_._2) + return compactLogs(logs).toArray + } catch { + case e: IOException => + // Another process using `FileStreamSink` may delete the batch files when + // `StreamFileCatalog` are reading. However, it only happens when a compaction is + // deleting old files. If so, let's try the next compaction batch and we should find it. + // Otherwise, this is a real IO issue and we should throw it. + latestId = nextCompactionBatchId(latestId, compactInterval) + get(latestId).getOrElse { + throw e + } + } + } else { + return Array.empty + } + } + Array.empty + } + + /** + * Compacts all logs before `batchId` plus the provided `logs`, and writes them into the + * corresponding `batchId` file. It will delete expired files as well if enabled. + */ + private def compact(batchId: Long, logs: Seq[SinkFileStatus]): Boolean = { + val validBatches = getValidBatchesBeforeCompactionBatch(batchId, compactInterval) + val allLogs = validBatches.flatMap(batchId => get(batchId)).flatten ++ logs + if (super.add(batchId, compactLogs(allLogs))) { + if (isDeletingExpiredLog) { + deleteExpiredLog(batchId) + } + true + } else { + // Return false as there is another writer. + false + } + } + + /** + * Since all logs before `compactionBatchId` are compacted and written into the + * `compactionBatchId` log file, they can be removed. However, due to the eventual consistency of + * S3, the compaction file may not be seen by other processes at once. So we only delete files + * created `fileCleanupDelayMs` milliseconds ago. + */ + private def deleteExpiredLog(compactionBatchId: Long): Unit = { + val expiredTime = System.currentTimeMillis() - fileCleanupDelayMs + fileManager.list(metadataPath, new PathFilter { + override def accept(path: Path): Boolean = { + try { + val batchId = getBatchIdFromFileName(path.getName) + batchId < compactionBatchId + } catch { + case _: NumberFormatException => + false + } + } + }).foreach { f => + if (f.getModificationTime <= expiredTime) { + fileManager.delete(f.getPath) + } + } + } +} + +object FileStreamSinkLog { + val VERSION = "v1" + val COMPACT_FILE_SUFFIX = ".compact" + val DELETE_ACTION = "delete" + val ADD_ACTION = "add" + + def getBatchIdFromFileName(fileName: String): Long = { + fileName.stripSuffix(COMPACT_FILE_SUFFIX).toLong + } + + /** + * Returns if this is a compaction batch. FileStreamSinkLog will compact old logs every + * `compactInterval` commits. + * + * E.g., if `compactInterval` is 3, then 2, 5, 8, ... are all compaction batches. + */ + def isCompactionBatch(batchId: Long, compactInterval: Int): Boolean = { + (batchId + 1) % compactInterval == 0 + } + + /** + * Returns all valid batches before the specified `compactionBatchId`. They contain all logs we + * need to do a new compaction. + * + * E.g., if `compactInterval` is 3 and `compactionBatchId` is 5, this method should returns + * `Seq(2, 3, 4)` (Note: it includes the previous compaction batch 2). + */ + def getValidBatchesBeforeCompactionBatch( + compactionBatchId: Long, + compactInterval: Int): Seq[Long] = { + assert(isCompactionBatch(compactionBatchId, compactInterval), + s"$compactionBatchId is not a compaction batch") + (math.max(0, compactionBatchId - compactInterval)) until compactionBatchId + } + + /** + * Returns all necessary logs before `batchId` (inclusive). If `batchId` is a compaction, just + * return itself. Otherwise, it will find the previous compaction batch and return all batches + * between it and `batchId`. + */ + def getAllValidBatches(batchId: Long, compactInterval: Long): Seq[Long] = { + assert(batchId >= 0) + val start = math.max(0, (batchId + 1) / compactInterval * compactInterval - 1) + start to batchId + } + + /** + * Removes all deleted files from logs. It assumes once one file is deleted, it won't be added to + * the log in future. + */ + def compactLogs(logs: Seq[SinkFileStatus]): Seq[SinkFileStatus] = { + val deletedFiles = logs.filter(_.action == DELETE_ACTION).map(_.path).toSet + if (deletedFiles.isEmpty) { + logs + } else { + logs.filter(f => !deletedFiles.contains(f.path)) + } + } + + /** + * Returns the next compaction batch id after `batchId`. + */ + def nextCompactionBatchId(batchId: Long, compactInterval: Long): Long = { + (batchId + compactInterval + 1) / compactInterval * compactInterval - 1 + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala index 9663fee18d364..b52f7a28b408a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala @@ -51,8 +51,8 @@ class HDFSMetadataLog[T: ClassTag](sqlContext: SQLContext, path: String) import HDFSMetadataLog._ - private val metadataPath = new Path(path) - private val fileManager = createFileManager() + val metadataPath = new Path(path) + protected val fileManager = createFileManager() if (!fileManager.exists(metadataPath)) { fileManager.mkdirs(metadataPath) @@ -62,7 +62,21 @@ class HDFSMetadataLog[T: ClassTag](sqlContext: SQLContext, path: String) * A `PathFilter` to filter only batch files */ private val batchFilesFilter = new PathFilter { - override def accept(path: Path): Boolean = try { + override def accept(path: Path): Boolean = isBatchFile(path) + } + + private val serializer = new JavaSerializer(sqlContext.sparkContext.conf).newInstance() + + protected def batchIdToPath(batchId: Long): Path = { + new Path(metadataPath, batchId.toString) + } + + protected def pathToBatchId(path: Path) = { + path.getName.toLong + } + + protected def isBatchFile(path: Path) = { + try { path.getName.toLong true } catch { @@ -70,18 +84,19 @@ class HDFSMetadataLog[T: ClassTag](sqlContext: SQLContext, path: String) } } - private val serializer = new JavaSerializer(sqlContext.sparkContext.conf).newInstance() + protected def serialize(metadata: T): Array[Byte] = { + JavaUtils.bufferToArray(serializer.serialize(metadata)) + } - private def batchFile(batchId: Long): Path = { - new Path(metadataPath, batchId.toString) + protected def deserialize(bytes: Array[Byte]): T = { + serializer.deserialize[T](ByteBuffer.wrap(bytes)) } override def add(batchId: Long, metadata: T): Boolean = { get(batchId).map(_ => false).getOrElse { // Only write metadata when the batch has not yet been written. - val buffer = serializer.serialize(metadata) try { - writeBatch(batchId, JavaUtils.bufferToArray(buffer)) + writeBatch(batchId, serialize(metadata)) true } catch { case e: IOException if "java.lang.InterruptedException" == e.getMessage => @@ -113,8 +128,8 @@ class HDFSMetadataLog[T: ClassTag](sqlContext: SQLContext, path: String) try { // Try to commit the batch // It will fail if there is an existing file (someone has committed the batch) - logDebug(s"Attempting to write log #${batchFile(batchId)}") - fileManager.rename(tempPath, batchFile(batchId)) + logDebug(s"Attempting to write log #${batchIdToPath(batchId)}") + fileManager.rename(tempPath, batchIdToPath(batchId)) return } catch { case e: IOException if isFileAlreadyExistsException(e) => @@ -158,11 +173,11 @@ class HDFSMetadataLog[T: ClassTag](sqlContext: SQLContext, path: String) } override def get(batchId: Long): Option[T] = { - val batchMetadataFile = batchFile(batchId) + val batchMetadataFile = batchIdToPath(batchId) if (fileManager.exists(batchMetadataFile)) { val input = fileManager.open(batchMetadataFile) val bytes = IOUtils.toByteArray(input) - Some(serializer.deserialize[T](ByteBuffer.wrap(bytes))) + Some(deserialize(bytes)) } else { logDebug(s"Unable to find batch $batchMetadataFile") None @@ -172,7 +187,7 @@ class HDFSMetadataLog[T: ClassTag](sqlContext: SQLContext, path: String) override def get(startId: Option[Long], endId: Option[Long]): Array[(Long, T)] = { val files = fileManager.list(metadataPath, batchFilesFilter) val batchIds = files - .map(_.getPath.getName.toLong) + .map(f => pathToBatchId(f.getPath)) .filter { batchId => (endId.isEmpty || batchId <= endId.get) && (startId.isEmpty || batchId >= startId.get) } @@ -184,7 +199,7 @@ class HDFSMetadataLog[T: ClassTag](sqlContext: SQLContext, path: String) override def getLatest(): Option[(Long, T)] = { val batchIds = fileManager.list(metadataPath, batchFilesFilter) - .map(_.getPath.getName.toLong) + .map(f => pathToBatchId(f.getPath)) .sorted .reverse for (batchId <- batchIds) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamFileCatalog.scala index b1f93a9159d50..95b5129351ff4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamFileCatalog.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.types.StructType class StreamFileCatalog(sqlContext: SQLContext, path: Path) extends FileCatalog with Logging { val metadataDirectory = new Path(path, FileStreamSink.metadataDir) logInfo(s"Reading streaming file log from $metadataDirectory") - val metadataLog = new HDFSMetadataLog[Seq[String]](sqlContext, metadataDirectory.toUri.toString) + val metadataLog = new FileStreamSinkLog(sqlContext, metadataDirectory.toUri.toString) val fs = path.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) override def paths: Seq[Path] = path :: Nil @@ -53,6 +53,6 @@ class StreamFileCatalog(sqlContext: SQLContext, path: Path) extends FileCatalog override def refresh(): Unit = {} override def allFiles(): Seq[FileStatus] = { - fs.listStatus(metadataLog.get(None, None).flatMap(_._2).map(new Path(_))) + metadataLog.allFiles().map(_.toFileStatus) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 4ae8278a9d767..80e2c1986d758 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.internal import java.util.{NoSuchElementException, Properties} +import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ import scala.collection.immutable @@ -463,6 +464,27 @@ object SQLConf { .booleanConf .createWithDefault(false) + val FILE_SINK_LOG_DELETION = SQLConfigBuilder("spark.sql.streaming.fileSink.log.deletion") + .internal() + .doc("Whether to delete the expired log files in file stream sink.") + .booleanConf + .createWithDefault(true) + + val FILE_SINK_LOG_COMPACT_INTERVAL = + SQLConfigBuilder("spark.sql.streaming.fileSink.log.compactInterval") + .internal() + .doc("Number of log files after which all the previous files " + + "are compacted into the next log file.") + .intConf + .createWithDefault(10) + + val FILE_SINK_LOG_CLEANUP_DELAY = + SQLConfigBuilder("spark.sql.streaming.fileSink.log.cleanupDelay") + .internal() + .doc("How long in milliseconds a file is guaranteed to be visible for all readers.") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefault(60 * 1000L) // 10 minutes + object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" val EXTERNAL_SORT = "spark.sql.planner.externalSort" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala new file mode 100644 index 0000000000000..70c2a82990ba9 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala @@ -0,0 +1,270 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import java.nio.charset.StandardCharsets.UTF_8 + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext + +class FileStreamSinkLogSuite extends SparkFunSuite with SharedSQLContext { + + import FileStreamSinkLog._ + + test("getBatchIdFromFileName") { + assert(1234L === getBatchIdFromFileName("1234")) + assert(1234L === getBatchIdFromFileName("1234.compact")) + intercept[NumberFormatException] { + FileStreamSinkLog.getBatchIdFromFileName("1234a") + } + } + + test("isCompactionBatch") { + assert(false === isCompactionBatch(0, compactInterval = 3)) + assert(false === isCompactionBatch(1, compactInterval = 3)) + assert(true === isCompactionBatch(2, compactInterval = 3)) + assert(false === isCompactionBatch(3, compactInterval = 3)) + assert(false === isCompactionBatch(4, compactInterval = 3)) + assert(true === isCompactionBatch(5, compactInterval = 3)) + } + + test("nextCompactionBatchId") { + assert(2 === nextCompactionBatchId(0, compactInterval = 3)) + assert(2 === nextCompactionBatchId(1, compactInterval = 3)) + assert(5 === nextCompactionBatchId(2, compactInterval = 3)) + assert(5 === nextCompactionBatchId(3, compactInterval = 3)) + assert(5 === nextCompactionBatchId(4, compactInterval = 3)) + assert(8 === nextCompactionBatchId(5, compactInterval = 3)) + } + + test("getValidBatchesBeforeCompactionBatch") { + intercept[AssertionError] { + getValidBatchesBeforeCompactionBatch(0, compactInterval = 3) + } + intercept[AssertionError] { + getValidBatchesBeforeCompactionBatch(1, compactInterval = 3) + } + assert(Seq(0, 1) === getValidBatchesBeforeCompactionBatch(2, compactInterval = 3)) + intercept[AssertionError] { + getValidBatchesBeforeCompactionBatch(3, compactInterval = 3) + } + intercept[AssertionError] { + getValidBatchesBeforeCompactionBatch(4, compactInterval = 3) + } + assert(Seq(2, 3, 4) === getValidBatchesBeforeCompactionBatch(5, compactInterval = 3)) + } + + test("getAllValidBatches") { + assert(Seq(0) === getAllValidBatches(0, compactInterval = 3)) + assert(Seq(0, 1) === getAllValidBatches(1, compactInterval = 3)) + assert(Seq(2) === getAllValidBatches(2, compactInterval = 3)) + assert(Seq(2, 3) === getAllValidBatches(3, compactInterval = 3)) + assert(Seq(2, 3, 4) === getAllValidBatches(4, compactInterval = 3)) + assert(Seq(5) === getAllValidBatches(5, compactInterval = 3)) + assert(Seq(5, 6) === getAllValidBatches(6, compactInterval = 3)) + assert(Seq(5, 6, 7) === getAllValidBatches(7, compactInterval = 3)) + assert(Seq(8) === getAllValidBatches(8, compactInterval = 3)) + } + + test("compactLogs") { + val logs = Seq( + newFakeSinkFileStatus("/a/b/x", FileStreamSinkLog.ADD_ACTION), + newFakeSinkFileStatus("/a/b/y", FileStreamSinkLog.ADD_ACTION), + newFakeSinkFileStatus("/a/b/z", FileStreamSinkLog.ADD_ACTION)) + assert(logs === compactLogs(logs)) + + val logs2 = Seq( + newFakeSinkFileStatus("/a/b/m", FileStreamSinkLog.ADD_ACTION), + newFakeSinkFileStatus("/a/b/n", FileStreamSinkLog.ADD_ACTION), + newFakeSinkFileStatus("/a/b/z", FileStreamSinkLog.DELETE_ACTION)) + assert(logs.dropRight(1) ++ logs2.dropRight(1) === compactLogs(logs ++ logs2)) + } + + test("serialize") { + withFileStreamSinkLog { sinkLog => + val logs = Seq( + SinkFileStatus( + path = "/a/b/x", + size = 100L, + isDir = false, + modificationTime = 1000L, + blockReplication = 1, + blockSize = 10000L, + action = FileStreamSinkLog.ADD_ACTION), + SinkFileStatus( + path = "/a/b/y", + size = 200L, + isDir = false, + modificationTime = 2000L, + blockReplication = 2, + blockSize = 20000L, + action = FileStreamSinkLog.DELETE_ACTION), + SinkFileStatus( + path = "/a/b/z", + size = 300L, + isDir = false, + modificationTime = 3000L, + blockReplication = 3, + blockSize = 30000L, + action = FileStreamSinkLog.ADD_ACTION)) + + // scalastyle:off + val expected = s"""${FileStreamSinkLog.VERSION} + |{"path":"/a/b/x","size":100,"isDir":false,"modificationTime":1000,"blockReplication":1,"blockSize":10000,"action":"add"} + |{"path":"/a/b/y","size":200,"isDir":false,"modificationTime":2000,"blockReplication":2,"blockSize":20000,"action":"delete"} + |{"path":"/a/b/z","size":300,"isDir":false,"modificationTime":3000,"blockReplication":3,"blockSize":30000,"action":"add"}""".stripMargin + // scalastyle:on + assert(expected === new String(sinkLog.serialize(logs), UTF_8)) + + assert(FileStreamSinkLog.VERSION === new String(sinkLog.serialize(Nil), UTF_8)) + } + } + + test("deserialize") { + withFileStreamSinkLog { sinkLog => + // scalastyle:off + val logs = s"""${FileStreamSinkLog.VERSION} + |{"path":"/a/b/x","size":100,"isDir":false,"modificationTime":1000,"blockReplication":1,"blockSize":10000,"action":"add"} + |{"path":"/a/b/y","size":200,"isDir":false,"modificationTime":2000,"blockReplication":2,"blockSize":20000,"action":"delete"} + |{"path":"/a/b/z","size":300,"isDir":false,"modificationTime":3000,"blockReplication":3,"blockSize":30000,"action":"add"}""".stripMargin + // scalastyle:on + + val expected = Seq( + SinkFileStatus( + path = "/a/b/x", + size = 100L, + isDir = false, + modificationTime = 1000L, + blockReplication = 1, + blockSize = 10000L, + action = FileStreamSinkLog.ADD_ACTION), + SinkFileStatus( + path = "/a/b/y", + size = 200L, + isDir = false, + modificationTime = 2000L, + blockReplication = 2, + blockSize = 20000L, + action = FileStreamSinkLog.DELETE_ACTION), + SinkFileStatus( + path = "/a/b/z", + size = 300L, + isDir = false, + modificationTime = 3000L, + blockReplication = 3, + blockSize = 30000L, + action = FileStreamSinkLog.ADD_ACTION)) + + assert(expected === sinkLog.deserialize(logs.getBytes(UTF_8))) + + assert(Nil === sinkLog.deserialize(FileStreamSinkLog.VERSION.getBytes(UTF_8))) + } + } + + test("batchIdToPath") { + withSQLConf(SQLConf.FILE_SINK_LOG_COMPACT_INTERVAL.key -> "3") { + withFileStreamSinkLog { sinkLog => + assert("0" === sinkLog.batchIdToPath(0).getName) + assert("1" === sinkLog.batchIdToPath(1).getName) + assert("2.compact" === sinkLog.batchIdToPath(2).getName) + assert("3" === sinkLog.batchIdToPath(3).getName) + assert("4" === sinkLog.batchIdToPath(4).getName) + assert("5.compact" === sinkLog.batchIdToPath(5).getName) + } + } + } + + test("compact") { + withSQLConf(SQLConf.FILE_SINK_LOG_COMPACT_INTERVAL.key -> "3") { + withFileStreamSinkLog { sinkLog => + for (batchId <- 0 to 10) { + sinkLog.add( + batchId, + Seq(newFakeSinkFileStatus("/a/b/" + batchId, FileStreamSinkLog.ADD_ACTION))) + val expectedFiles = (0 to batchId).map { + id => newFakeSinkFileStatus("/a/b/" + id, FileStreamSinkLog.ADD_ACTION) + } + assert(sinkLog.allFiles() === expectedFiles) + if (isCompactionBatch(batchId, 3)) { + // Since batchId is a compaction batch, the batch log file should contain all logs + assert(sinkLog.get(batchId).getOrElse(Nil) === expectedFiles) + } + } + } + } + } + + test("delete expired file") { + // Set FILE_SINK_LOG_CLEANUP_DELAY to 0 so that we can detect the deleting behaviour + // deterministically + withSQLConf( + SQLConf.FILE_SINK_LOG_COMPACT_INTERVAL.key -> "3", + SQLConf.FILE_SINK_LOG_CLEANUP_DELAY.key -> "0") { + withFileStreamSinkLog { sinkLog => + val fs = sinkLog.metadataPath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) + + def listBatchFiles(): Set[String] = { + fs.listStatus(sinkLog.metadataPath).map(_.getPath.getName).filter { fileName => + try { + getBatchIdFromFileName(fileName) + true + } catch { + case _: NumberFormatException => false + } + }.toSet + } + + sinkLog.add(0, Seq(newFakeSinkFileStatus("/a/b/0", FileStreamSinkLog.ADD_ACTION))) + assert(Set("0") === listBatchFiles()) + sinkLog.add(1, Seq(newFakeSinkFileStatus("/a/b/1", FileStreamSinkLog.ADD_ACTION))) + assert(Set("0", "1") === listBatchFiles()) + sinkLog.add(2, Seq(newFakeSinkFileStatus("/a/b/2", FileStreamSinkLog.ADD_ACTION))) + assert(Set("2.compact") === listBatchFiles()) + sinkLog.add(3, Seq(newFakeSinkFileStatus("/a/b/3", FileStreamSinkLog.ADD_ACTION))) + assert(Set("2.compact", "3") === listBatchFiles()) + sinkLog.add(4, Seq(newFakeSinkFileStatus("/a/b/4", FileStreamSinkLog.ADD_ACTION))) + assert(Set("2.compact", "3", "4") === listBatchFiles()) + sinkLog.add(5, Seq(newFakeSinkFileStatus("/a/b/5", FileStreamSinkLog.ADD_ACTION))) + assert(Set("5.compact") === listBatchFiles()) + } + } + } + + /** + * Create a fake SinkFileStatus using path and action. Most of tests don't care about other fields + * in SinkFileStatus. + */ + private def newFakeSinkFileStatus(path: String, action: String): SinkFileStatus = { + SinkFileStatus( + path = path, + size = 100L, + isDir = false, + modificationTime = 100L, + blockReplication = 1, + blockSize = 100L, + action = action) + } + + private def withFileStreamSinkLog(f: FileStreamSinkLog => Unit): Unit = { + withTempDir { file => + val sinkLog = new FileStreamSinkLog(sqlContext, file.getCanonicalPath) + f(sinkLog) + } + } +} From e7791c4f69aaa150e6ddb30b6d4ba2b0ea3c7807 Mon Sep 17 00:00:00 2001 From: "Sheamus K. Parkes" Date: Wed, 20 Apr 2016 13:45:14 -0700 Subject: [PATCH 64/69] [SPARK-13842] [PYSPARK] pyspark.sql.types.StructType accessor enhancements ## What changes were proposed in this pull request? Expand the possible ways to interact with the contents of a `pyspark.sql.types.StructType` instance. - Iterating a `StructType` will iterate its fields - `[field.name for field in my_structtype]` - Indexing with a string will return a field by name - `my_structtype['my_field_name']` - Indexing with an integer will return a field by position - `my_structtype[0]` - Indexing with a slice will return a new `StructType` with just the chosen fields: - `my_structtype[1:3]` - The length is the number of fields (should also provide "truthiness" for free) - `len(my_structtype) == 2` ## How was this patch tested? Extended the unit test coverage in the accompanying `tests.py`. Author: Sheamus K. Parkes Closes #12251 from skparkes/pyspark-structtype-enhance. --- python/pyspark/sql/tests.py | 23 +++++++++++++++---- python/pyspark/sql/types.py | 44 ++++++++++++++++++++++++++++++++----- 2 files changed, 58 insertions(+), 9 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 1e864b4cd1bd7..3b1b2948e9f94 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -802,11 +802,26 @@ def test_struct_type(self): self.assertNotEqual(struct1, struct2) # Catch exception raised during improper construction - try: + with self.assertRaises(ValueError): struct1 = StructType().add("name") - self.assertEqual(1, 0) - except ValueError: - self.assertEqual(1, 1) + + struct1 = StructType().add("f1", StringType(), True).add("f2", StringType(), True, None) + for field in struct1: + self.assertIsInstance(field, StructField) + + struct1 = StructType().add("f1", StringType(), True).add("f2", StringType(), True, None) + self.assertEqual(len(struct1), 2) + + struct1 = StructType().add("f1", StringType(), True).add("f2", StringType(), True, None) + self.assertIs(struct1["f1"], struct1.fields[0]) + self.assertIs(struct1[0], struct1.fields[0]) + self.assertEqual(struct1[0:1], StructType(struct1.fields[0:1])) + with self.assertRaises(KeyError): + not_a_field = struct1["f9"] + with self.assertRaises(IndexError): + not_a_field = struct1[9] + with self.assertRaises(TypeError): + not_a_field = struct1[9.9] def test_metadata_null(self): from pyspark.sql.types import StructType, StringType, StructField diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 734c1533a24bc..f7cd4b80ca91d 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -442,6 +442,15 @@ class StructType(DataType): """Struct type, consisting of a list of :class:`StructField`. This is the data type representing a :class:`Row`. + + Iterating a :class:`StructType` will iterate its :class:`StructField`s. + A contained :class:`StructField` can be accessed by name or position. + + >>> struct1 = StructType([StructField("f1", StringType(), True)]) + >>> struct1["f1"] + StructField(f1,StringType,true) + >>> struct1[0] + StructField(f1,StringType,true) """ def __init__(self, fields=None): """ @@ -463,7 +472,7 @@ def __init__(self, fields=None): self.names = [f.name for f in fields] assert all(isinstance(f, StructField) for f in fields),\ "fields should be a list of StructField" - self._needSerializeAnyField = any(f.needConversion() for f in self.fields) + self._needSerializeAnyField = any(f.needConversion() for f in self) def add(self, field, data_type=None, nullable=True, metadata=None): """ @@ -508,19 +517,44 @@ def add(self, field, data_type=None, nullable=True, metadata=None): data_type_f = data_type self.fields.append(StructField(field, data_type_f, nullable, metadata)) self.names.append(field) - self._needSerializeAnyField = any(f.needConversion() for f in self.fields) + self._needSerializeAnyField = any(f.needConversion() for f in self) return self + def __iter__(self): + """Iterate the fields""" + return iter(self.fields) + + def __len__(self): + """Return the number of fields.""" + return len(self.fields) + + def __getitem__(self, key): + """Access fields by name or slice.""" + if isinstance(key, str): + for field in self: + if field.name == key: + return field + raise KeyError('No StructField named {0}'.format(key)) + elif isinstance(key, int): + try: + return self.fields[key] + except IndexError: + raise IndexError('StructType index out of range') + elif isinstance(key, slice): + return StructType(self.fields[key]) + else: + raise TypeError('StructType keys should be strings, integers or slices') + def simpleString(self): - return 'struct<%s>' % (','.join(f.simpleString() for f in self.fields)) + return 'struct<%s>' % (','.join(f.simpleString() for f in self)) def __repr__(self): return ("StructType(List(%s))" % - ",".join(str(field) for field in self.fields)) + ",".join(str(field) for field in self)) def jsonValue(self): return {"type": self.typeName(), - "fields": [f.jsonValue() for f in self.fields]} + "fields": [f.jsonValue() for f in self]} @classmethod def fromJson(cls, json): From fd82681945447d97df368bcdcd112b08bd541cca Mon Sep 17 00:00:00 2001 From: Subhobrata Dey Date: Wed, 20 Apr 2016 14:26:07 -0700 Subject: [PATCH 65/69] [SPARK-14749][SQL, TESTS] PlannerSuite failed when it run individually ## What changes were proposed in this pull request? 3 testcases namely, ``` "count is partially aggregated" "count distinct is partially aggregated" "mixed aggregates are partially aggregated" ``` were failing when running PlannerSuite individually. The PR provides a fix for this. ## How was this patch tested? unit tests (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Author: Subhobrata Dey Closes #12532 from sbcd90/plannersuitetestsfix. --- .../org/apache/spark/sql/execution/QueryExecution.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index ddcae0fe07533..35228643a5f49 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -51,7 +51,10 @@ class QueryExecution(val sqlContext: SQLContext, val logical: LogicalPlan) { } } - lazy val analyzed: LogicalPlan = sqlContext.sessionState.analyzer.execute(logical) + lazy val analyzed: LogicalPlan = { + SQLContext.setActive(sqlContext) + sqlContext.sessionState.analyzer.execute(logical) + } lazy val withCachedData: LogicalPlan = { assertAnalyzed() From b28fe448d91330afc84e369c548ab0a5de91e9ac Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 20 Apr 2016 16:29:26 -0700 Subject: [PATCH 66/69] [SPARK-14770][SQL] Remove unused queries in hive module test resources ## What changes were proposed in this pull request? We currently have five folders in queries: clientcompare, clientnegative, clientpositive, negative, and positive. Only clientpositive is used. We can remove the rest. ## How was this patch tested? N/A - removing unused test resources. Author: Reynold Xin Closes #12540 from rxin/SPARK-14770. --- .../clientcompare/vectorized_math_funcs.q | 43 ----------- .../clientcompare/vectorized_math_funcs_00.qv | 1 - .../clientcompare/vectorized_math_funcs_01.qv | 1 - .../add_partition_with_whitelist.q | 8 -- .../test/queries/clientnegative/addpart1.q | 11 --- .../alter_concatenate_indexed_table.q | 16 ---- .../queries/clientnegative/alter_non_native.q | 6 -- .../alter_partition_coltype_2columns.q | 11 --- .../alter_partition_coltype_invalidcolname.q | 12 --- .../alter_partition_coltype_invalidtype.q | 11 --- .../alter_partition_invalidspec.q | 8 -- .../clientnegative/alter_partition_nodrop.q | 9 --- .../alter_partition_nodrop_table.q | 9 --- .../clientnegative/alter_partition_offline.q | 11 --- .../alter_partition_with_whitelist.q | 9 --- .../alter_rename_partition_failure.q | 6 -- .../alter_rename_partition_failure2.q | 6 -- .../alter_rename_partition_failure3.q | 6 -- .../alter_table_add_partition.q | 5 -- .../clientnegative/alter_table_wrong_regex.q | 7 -- .../alter_view_as_select_not_exist.q | 4 - .../alter_view_as_select_with_partition.q | 12 --- .../clientnegative/alter_view_failure.q | 3 - .../clientnegative/alter_view_failure2.q | 8 -- .../clientnegative/alter_view_failure3.q | 2 - .../clientnegative/alter_view_failure4.q | 8 -- .../clientnegative/alter_view_failure5.q | 8 -- .../clientnegative/alter_view_failure6.q | 11 --- .../clientnegative/alter_view_failure7.q | 8 -- .../clientnegative/alter_view_failure8.q | 3 - .../clientnegative/alter_view_failure9.q | 7 -- .../src/test/queries/clientnegative/altern1.q | 4 - .../queries/clientnegative/ambiguous_col.q | 1 - .../queries/clientnegative/ambiguous_col0.q | 2 - .../queries/clientnegative/ambiguous_col1.q | 3 - .../queries/clientnegative/ambiguous_col2.q | 3 - .../src/test/queries/clientnegative/analyze.q | 1 - .../test/queries/clientnegative/analyze1.q | 1 - .../clientnegative/analyze_non_existent_tbl.q | 1 - .../queries/clientnegative/analyze_view.q | 6 -- .../test/queries/clientnegative/archive1.q | 11 --- .../test/queries/clientnegative/archive2.q | 10 --- .../test/queries/clientnegative/archive3.q | 5 -- .../test/queries/clientnegative/archive4.q | 5 -- .../test/queries/clientnegative/archive5.q | 5 -- .../queries/clientnegative/archive_corrupt.q | 18 ----- .../queries/clientnegative/archive_insert1.q | 13 ---- .../queries/clientnegative/archive_insert2.q | 13 ---- .../queries/clientnegative/archive_insert3.q | 13 ---- .../queries/clientnegative/archive_insert4.q | 15 ---- .../queries/clientnegative/archive_multi1.q | 13 ---- .../queries/clientnegative/archive_multi2.q | 12 --- .../queries/clientnegative/archive_multi3.q | 13 ---- .../queries/clientnegative/archive_multi4.q | 13 ---- .../queries/clientnegative/archive_multi5.q | 13 ---- .../queries/clientnegative/archive_multi6.q | 13 ---- .../queries/clientnegative/archive_multi7.q | 12 --- .../clientnegative/archive_partspec1.q | 10 --- .../clientnegative/archive_partspec2.q | 10 --- .../clientnegative/archive_partspec3.q | 10 --- .../clientnegative/archive_partspec4.q | 10 --- .../clientnegative/archive_partspec5.q | 10 --- .../clientnegative/authorization_addjar.q | 7 -- .../authorization_addpartition.q | 10 --- .../authorization_alter_db_owner.q | 11 --- .../authorization_alter_db_owner_default.q | 8 -- .../authorization_cannot_create_all_role.q | 6 -- ...authorization_cannot_create_default_role.q | 6 -- .../authorization_cannot_create_none_role.q | 6 -- .../authorization_caseinsensitivity.q | 17 ----- .../authorization_create_func1.q | 7 -- .../authorization_create_func2.q | 8 -- .../authorization_create_macro1.q | 8 -- .../authorization_create_role_no_admin.q | 3 - .../clientnegative/authorization_createview.q | 10 --- .../clientnegative/authorization_ctas.q | 10 --- .../authorization_desc_table_nosel.q | 14 ---- .../clientnegative/authorization_dfs.q | 7 -- .../authorization_disallow_transform.q | 3 - .../authorization_drop_db_cascade.q | 22 ------ .../authorization_drop_db_empty.q | 27 ------- .../authorization_drop_role_no_admin.q | 10 --- .../authorization_droppartition.q | 11 --- .../clientnegative/authorization_fail_1.q | 7 -- .../clientnegative/authorization_fail_2.q | 7 -- .../clientnegative/authorization_fail_3.q | 12 --- .../clientnegative/authorization_fail_4.q | 15 ---- .../clientnegative/authorization_fail_5.q | 20 ----- .../clientnegative/authorization_fail_6.q | 6 -- .../clientnegative/authorization_fail_7.q | 17 ----- .../authorization_fail_create_db.q | 5 -- .../authorization_fail_drop_db.q | 5 -- .../authorization_grant_table_allpriv.q | 14 ---- .../authorization_grant_table_dup.q | 16 ---- .../authorization_grant_table_fail1.q | 11 --- .../authorization_grant_table_fail_nogrant.q | 14 ---- .../authorization_insert_noinspriv.q | 11 --- .../authorization_insert_noselectpriv.q | 11 --- .../authorization_invalid_priv_v1.q | 6 -- .../authorization_invalid_priv_v2.q | 5 -- ...authorization_not_owner_alter_tab_rename.q | 10 --- ...horization_not_owner_alter_tab_serdeprop.q | 10 --- .../authorization_not_owner_drop_tab.q | 11 --- .../authorization_not_owner_drop_view.q | 11 --- .../clientnegative/authorization_part.q | 37 ---------- .../authorization_priv_current_role_neg.q | 29 -------- .../authorization_public_create.q | 1 - .../authorization_public_drop.q | 1 - .../authorization_revoke_table_fail1.q | 14 ---- .../authorization_revoke_table_fail2.q | 18 ----- .../authorization_role_cycles1.q | 12 --- .../authorization_role_cycles2.q | 24 ------ .../clientnegative/authorization_role_grant.q | 22 ------ .../authorization_rolehierarchy_privs.q | 74 ------------------- .../clientnegative/authorization_select.q | 9 --- .../authorization_select_view.q | 11 --- .../authorization_set_role_neg1.q | 6 -- .../authorization_set_role_neg2.q | 16 ---- .../authorization_show_parts_nosel.q | 10 --- ...horization_show_role_principals_no_admin.q | 3 - .../authorization_show_role_principals_v1.q | 2 - .../authorization_show_roles_no_admin.q | 3 - .../clientnegative/authorization_truncate.q | 9 --- .../authorization_uri_add_partition.q | 10 --- .../authorization_uri_alterpart_loc.q | 16 ---- .../authorization_uri_altertab_setloc.q | 13 ---- .../authorization_uri_create_table1.q | 11 --- .../authorization_uri_create_table_ext.q | 11 --- .../authorization_uri_createdb.q | 12 --- .../clientnegative/authorization_uri_export.q | 22 ------ .../clientnegative/authorization_uri_import.q | 25 ------- .../clientnegative/authorization_uri_index.q | 13 ---- .../clientnegative/authorization_uri_insert.q | 14 ---- .../authorization_uri_insert_local.q | 14 ---- .../authorization_uri_load_data.q | 11 --- .../clientnegative/authorize_create_tbl.q | 10 --- .../clientnegative/authorize_grant_public.q | 1 - .../clientnegative/authorize_revoke_public.q | 1 - .../test/queries/clientnegative/autolocal1.q | 15 ---- .../queries/clientnegative/bad_exec_hooks.q | 6 -- .../queries/clientnegative/bad_indextype.q | 1 - .../clientnegative/bad_sample_clause.q | 6 -- .../clientnegative/bucket_mapjoin_mismatch1.q | 42 ----------- .../bucket_mapjoin_wrong_table_metadata_1.q | 20 ----- .../bucket_mapjoin_wrong_table_metadata_2.q | 24 ------ .../clientnegative/cachingprintstream.q | 8 -- .../cluster_tasklog_retrieval.q | 6 -- .../clientnegative/clusterbydistributeby.q | 8 -- .../queries/clientnegative/clusterbyorderby.q | 5 -- .../queries/clientnegative/clusterbysortby.q | 8 -- .../test/queries/clientnegative/clustern2.q | 3 - .../test/queries/clientnegative/clustern3.q | 2 - .../test/queries/clientnegative/clustern4.q | 2 - .../column_change_skewedcol_type1.q | 5 -- .../queries/clientnegative/column_rename1.q | 6 -- .../queries/clientnegative/column_rename2.q | 6 -- .../queries/clientnegative/column_rename3.q | 1 - .../queries/clientnegative/column_rename4.q | 6 -- .../queries/clientnegative/column_rename5.q | 6 -- .../clientnegative/columnstats_partlvl_dp.q | 16 ---- .../columnstats_partlvl_incorrect_num_keys.q | 16 ---- .../columnstats_partlvl_invalid_values.q | 16 ---- ...columnstats_partlvl_multiple_part_clause.q | 16 ---- .../clientnegative/columnstats_tbllvl.q | 22 ------ .../columnstats_tbllvl_complex_type.q | 17 ----- .../columnstats_tbllvl_incorrect_column.q | 22 ------ .../clientnegative/compare_double_bigint.q | 5 -- .../clientnegative/compare_string_bigint.q | 5 -- .../clientnegative/compile_processor.q | 8 -- .../clientnegative/compute_stats_long.q | 7 -- .../create_function_nonexistent_class.q | 1 - .../create_function_nonexistent_db.q | 1 - .../create_function_nonudf_class.q | 1 - .../create_insert_outputformat.q | 11 --- .../clientnegative/create_or_replace_view1.q | 6 -- .../clientnegative/create_or_replace_view2.q | 6 -- .../clientnegative/create_or_replace_view3.q | 3 - .../clientnegative/create_or_replace_view4.q | 5 -- .../clientnegative/create_or_replace_view5.q | 5 -- .../clientnegative/create_or_replace_view6.q | 5 -- .../clientnegative/create_or_replace_view7.q | 7 -- .../clientnegative/create_or_replace_view8.q | 5 -- ..._skewed_table_col_name_value_no_mismatch.q | 3 - .../create_skewed_table_dup_col_name.q | 3 - ...te_skewed_table_failure_invalid_col_name.q | 4 - .../clientnegative/create_table_failure1.q | 1 - .../clientnegative/create_table_failure2.q | 1 - .../clientnegative/create_table_failure3.q | 1 - .../clientnegative/create_table_failure4.q | 1 - .../clientnegative/create_table_wrong_regex.q | 4 - .../clientnegative/create_udaf_failure.q | 6 -- .../create_unknown_genericudf.q | 1 - .../clientnegative/create_unknown_udf_udaf.q | 1 - .../clientnegative/create_view_failure1.q | 6 -- .../clientnegative/create_view_failure2.q | 6 -- .../clientnegative/create_view_failure3.q | 5 -- .../clientnegative/create_view_failure4.q | 5 -- .../clientnegative/create_view_failure5.q | 9 --- .../clientnegative/create_view_failure6.q | 6 -- .../clientnegative/create_view_failure7.q | 6 -- .../clientnegative/create_view_failure8.q | 6 -- .../clientnegative/create_view_failure9.q | 6 -- .../ql/src/test/queries/clientnegative/ctas.q | 5 -- .../queries/clientnegative/cte_recursion.q | 4 - .../clientnegative/cte_with_in_subquery.q | 1 - .../database_create_already_exists.q | 5 -- .../database_create_invalid_name.q | 4 - .../database_drop_does_not_exist.q | 4 - .../clientnegative/database_drop_not_empty.q | 8 -- .../database_drop_not_empty_restrict.q | 8 -- .../database_switch_does_not_exist.q | 4 - .../queries/clientnegative/date_literal2.q | 2 - .../queries/clientnegative/date_literal3.q | 2 - .../clientnegative/dbtxnmgr_nodblock.q | 6 -- .../clientnegative/dbtxnmgr_nodbunlock.q | 6 -- .../clientnegative/dbtxnmgr_notablelock.q | 6 -- .../clientnegative/dbtxnmgr_notableunlock.q | 6 -- .../src/test/queries/clientnegative/ddltime.q | 6 -- .../clientnegative/decimal_precision.q | 10 --- .../clientnegative/decimal_precision_1.q | 10 --- .../clientnegative/default_partition_name.q | 3 - .../test/queries/clientnegative/deletejar.q | 4 - .../queries/clientnegative/desc_failure1.q | 1 - .../queries/clientnegative/desc_failure2.q | 2 - .../queries/clientnegative/desc_failure3.q | 5 -- .../queries/clientnegative/describe_xpath1.q | 1 - .../queries/clientnegative/describe_xpath2.q | 1 - .../queries/clientnegative/describe_xpath3.q | 1 - .../queries/clientnegative/describe_xpath4.q | 1 - .../disallow_incompatible_type_change_on1.q | 17 ----- .../disallow_incompatible_type_change_on2.q | 6 -- .../clientnegative/drop_func_nonexistent.q | 3 - .../clientnegative/drop_function_failure.q | 3 - .../clientnegative/drop_index_failure.q | 3 - .../queries/clientnegative/drop_native_udf.q | 1 - .../clientnegative/drop_partition_failure.q | 11 --- .../drop_partition_filter_failure.q | 8 -- .../clientnegative/drop_table_failure1.q | 3 - .../clientnegative/drop_table_failure2.q | 3 - .../clientnegative/drop_table_failure3.q | 12 --- .../clientnegative/drop_view_failure1.q | 6 -- .../clientnegative/drop_view_failure2.q | 3 - .../duplicate_alias_in_transform.q | 1 - .../duplicate_alias_in_transform_schema.q | 1 - .../clientnegative/duplicate_insert1.q | 7 -- .../clientnegative/duplicate_insert2.q | 6 -- .../clientnegative/duplicate_insert3.q | 4 - .../test/queries/clientnegative/dyn_part1.q | 11 --- .../test/queries/clientnegative/dyn_part2.q | 11 --- .../test/queries/clientnegative/dyn_part3.q | 9 --- .../test/queries/clientnegative/dyn_part4.q | 7 -- .../clientnegative/dyn_part_empty.q.disabled | 24 ------ .../queries/clientnegative/dyn_part_max.q | 16 ---- .../clientnegative/dyn_part_max_per_node.q | 15 ---- .../dynamic_partitions_with_whitelist.q | 17 ----- ...hange_partition_neg_incomplete_partition.q | 12 --- .../exchange_partition_neg_partition_exists.q | 12 --- ...exchange_partition_neg_partition_exists2.q | 13 ---- ...exchange_partition_neg_partition_exists3.q | 13 ---- ...exchange_partition_neg_partition_missing.q | 6 -- .../exchange_partition_neg_table_missing.q | 2 - .../exchange_partition_neg_table_missing2.q | 8 -- .../exchange_partition_neg_test.q | 11 --- .../exim_00_unsupported_schema.q | 12 --- .../exim_01_nonpart_over_loaded.q | 24 ------ .../exim_02_all_part_over_overlap.q | 38 ---------- .../exim_03_nonpart_noncompat_colschema.q | 23 ------ .../exim_04_nonpart_noncompat_colnumber.q | 23 ------ .../exim_05_nonpart_noncompat_coltype.q | 23 ------ .../exim_06_nonpart_noncompat_storage.q | 23 ------ .../exim_07_nonpart_noncompat_ifof.q | 26 ------- .../exim_08_nonpart_noncompat_serde.q | 24 ------ .../exim_09_nonpart_noncompat_serdeparam.q | 28 ------- .../exim_10_nonpart_noncompat_bucketing.q | 24 ------ .../exim_11_nonpart_noncompat_sorting.q | 25 ------- .../clientnegative/exim_12_nonnative_export.q | 9 --- .../clientnegative/exim_13_nonnative_import.q | 24 ------ .../clientnegative/exim_14_nonpart_part.q | 25 ------- .../clientnegative/exim_15_part_nonpart.q | 25 ------- .../exim_16_part_noncompat_schema.q | 26 ------- .../exim_17_part_spec_underspec.q | 30 -------- .../exim_18_part_spec_missing.q | 30 -------- .../exim_19_external_over_existing.q | 23 ------ .../exim_20_managed_location_over_existing.q | 30 -------- .../exim_21_part_managed_external.q | 35 --------- .../clientnegative/exim_22_export_authfail.q | 14 ---- .../exim_23_import_exist_authfail.q | 22 ------ .../exim_24_import_part_authfail.q | 31 -------- .../exim_25_import_nonexist_authfail.q | 23 ------ .../test/queries/clientnegative/external1.q | 3 - .../test/queries/clientnegative/external2.q | 4 - .../clientnegative/fetchtask_ioexception.q | 7 -- .../file_with_header_footer_negative.q | 13 ---- .../clientnegative/fileformat_bad_class.q | 3 - .../clientnegative/fileformat_void_input.q | 8 -- .../clientnegative/fileformat_void_output.q | 6 -- .../queries/clientnegative/fs_default_name1.q | 2 - .../queries/clientnegative/fs_default_name2.q | 2 - .../clientnegative/genericFileFormat.q | 1 - .../groupby2_map_skew_multi_distinct.q | 14 ---- .../clientnegative/groupby2_multi_distinct.q | 13 ---- .../groupby3_map_skew_multi_distinct.q | 36 --------- .../clientnegative/groupby3_multi_distinct.q | 36 --------- .../queries/clientnegative/groupby_cube1.q | 4 - .../queries/clientnegative/groupby_cube2.q | 4 - .../clientnegative/groupby_grouping_id1.q | 4 - .../clientnegative/groupby_grouping_sets1.q | 5 -- .../clientnegative/groupby_grouping_sets2.q | 4 - .../clientnegative/groupby_grouping_sets3.q | 4 - .../clientnegative/groupby_grouping_sets4.q | 4 - .../clientnegative/groupby_grouping_sets5.q | 5 -- .../clientnegative/groupby_grouping_sets6.q | 8 -- .../clientnegative/groupby_grouping_sets7.q | 10 --- .../clientnegative/groupby_invalid_position.q | 4 - .../test/queries/clientnegative/groupby_key.q | 1 - .../queries/clientnegative/groupby_rollup1.q | 4 - .../queries/clientnegative/groupby_rollup2.q | 4 - .../src/test/queries/clientnegative/having1.q | 2 - .../clientnegative/illegal_partition_type.q | 7 -- .../clientnegative/illegal_partition_type2.q | 3 - .../clientnegative/illegal_partition_type3.q | 4 - .../clientnegative/illegal_partition_type4.q | 3 - .../clientnegative/index_bitmap_no_map_aggr.q | 7 -- .../index_compact_entry_limit.q | 12 --- .../clientnegative/index_compact_size_limit.q | 13 ---- .../src/test/queries/clientnegative/input1.q | 1 - .../src/test/queries/clientnegative/input2.q | 1 - .../src/test/queries/clientnegative/input4.q | 5 -- .../src/test/queries/clientnegative/input41.q | 5 -- .../queries/clientnegative/input_part0_neg.q | 3 - .../queries/clientnegative/insert_into1.q | 11 --- .../queries/clientnegative/insert_into2.q | 10 --- .../queries/clientnegative/insert_into3.q | 16 ---- .../queries/clientnegative/insert_into4.q | 16 ---- .../queries/clientnegative/insert_into5.q | 9 --- .../queries/clientnegative/insert_into6.q | 17 ----- .../clientnegative/insert_view_failure.q | 5 -- .../queries/clientnegative/insertexternal1.q | 8 -- .../insertover_dynapart_ifnotexists.q | 9 --- .../clientnegative/invalid_arithmetic_type.q | 3 - .../clientnegative/invalid_avg_syntax.q | 1 - .../invalid_cast_from_binary_1.q | 2 - .../invalid_cast_from_binary_2.q | 2 - .../invalid_cast_from_binary_3.q | 2 - .../invalid_cast_from_binary_4.q | 2 - .../invalid_cast_from_binary_5.q | 2 - .../invalid_cast_from_binary_6.q | 2 - .../clientnegative/invalid_cast_to_binary_1.q | 1 - .../clientnegative/invalid_cast_to_binary_2.q | 1 - .../clientnegative/invalid_cast_to_binary_3.q | 1 - .../clientnegative/invalid_cast_to_binary_4.q | 1 - .../clientnegative/invalid_cast_to_binary_5.q | 1 - .../clientnegative/invalid_cast_to_binary_6.q | 1 - .../clientnegative/invalid_char_length_1.q | 2 - .../clientnegative/invalid_char_length_2.q | 1 - .../clientnegative/invalid_char_length_3.q | 3 - .../queries/clientnegative/invalid_config1.q | 3 - .../queries/clientnegative/invalid_config2.q | 4 - .../clientnegative/invalid_create_tbl1.q | 9 --- .../clientnegative/invalid_create_tbl2.q | 1 - .../queries/clientnegative/invalid_mapjoin1.q | 1 - .../clientnegative/invalid_max_syntax.q | 1 - .../clientnegative/invalid_min_syntax.q | 1 - .../clientnegative/invalid_select_column.q | 4 - .../invalid_select_column_with_subquery.q | 4 - .../invalid_select_column_with_tablename.q | 4 - .../invalid_select_expression.q | 1 - .../clientnegative/invalid_std_syntax.q | 1 - .../invalid_stddev_samp_syntax.q | 1 - .../clientnegative/invalid_sum_syntax.q | 1 - .../queries/clientnegative/invalid_t_alter1.q | 2 - .../queries/clientnegative/invalid_t_alter2.q | 2 - .../clientnegative/invalid_t_create2.q | 1 - .../clientnegative/invalid_t_transform.q | 1 - .../queries/clientnegative/invalid_tbl_name.q | 1 - .../clientnegative/invalid_var_samp_syntax.q | 1 - .../clientnegative/invalid_varchar_length_1.q | 2 - .../clientnegative/invalid_varchar_length_2.q | 1 - .../clientnegative/invalid_varchar_length_3.q | 3 - .../clientnegative/invalid_variance_syntax.q | 1 - .../queries/clientnegative/invalidate_view1.q | 11 --- .../src/test/queries/clientnegative/join2.q | 5 -- .../src/test/queries/clientnegative/join28.q | 15 ---- .../src/test/queries/clientnegative/join29.q | 10 --- .../src/test/queries/clientnegative/join32.q | 14 ---- .../src/test/queries/clientnegative/join35.q | 18 ----- .../clientnegative/join_alt_syntax_comma_on.q | 3 - .../join_cond_unqual_ambiguous.q | 6 -- .../join_cond_unqual_ambiguous_vc.q | 5 -- .../clientnegative/join_nonexistent_part.q | 4 - .../src/test/queries/clientnegative/joinneg.q | 6 -- .../clientnegative/lateral_view_alias.q | 3 - .../clientnegative/lateral_view_join.q | 1 - .../queries/clientnegative/limit_partition.q | 7 -- .../clientnegative/limit_partition_stats.q | 18 ----- .../queries/clientnegative/line_terminator.q | 3 - .../clientnegative/load_exist_part_authfail.q | 4 - .../queries/clientnegative/load_non_native.q | 5 -- .../clientnegative/load_nonpart_authfail.q | 3 - .../clientnegative/load_part_authfail.q | 3 - .../queries/clientnegative/load_part_nospec.q | 2 - .../clientnegative/load_stored_as_dirs.q | 7 -- .../clientnegative/load_view_failure.q | 3 - .../clientnegative/load_wrong_fileformat.q | 6 -- .../load_wrong_fileformat_rc_seq.q | 6 -- .../load_wrong_fileformat_txt_seq.q | 6 -- .../clientnegative/load_wrong_noof_part.q | 3 - .../clientnegative/local_mapred_error_cache.q | 4 - .../test/queries/clientnegative/lockneg1.q | 10 --- .../test/queries/clientnegative/lockneg2.q | 6 -- .../test/queries/clientnegative/lockneg3.q | 9 --- .../test/queries/clientnegative/lockneg4.q | 12 --- .../test/queries/clientnegative/lockneg5.q | 2 - .../lockneg_query_tbl_in_locked_db.q | 17 ----- .../lockneg_try_db_lock_conflict.q | 6 -- .../lockneg_try_drop_locked_db.q | 8 -- .../lockneg_try_lock_db_in_use.q | 15 ---- .../clientnegative/macro_unused_parameter.q | 1 - .../clientnegative/mapreduce_stack_trace.q | 13 ---- .../mapreduce_stack_trace_hadoop20.q | 13 ---- .../mapreduce_stack_trace_turnoff.q | 13 ---- .../mapreduce_stack_trace_turnoff_hadoop20.q | 13 ---- .../queries/clientnegative/merge_negative_1.q | 3 - .../queries/clientnegative/merge_negative_2.q | 3 - .../queries/clientnegative/merge_negative_3.q | 6 -- .../clientnegative/minimr_broken_pipe.q | 4 - .../clientnegative/nested_complex_neg.q | 15 ---- .../queries/clientnegative/no_matching_udf.q | 1 - .../queries/clientnegative/nonkey_groupby.q | 1 - .../queries/clientnegative/nopart_insert.q | 7 -- .../test/queries/clientnegative/nopart_load.q | 5 -- .../queries/clientnegative/notable_alias4.q | 4 - .../clientnegative/orderby_invalid_position.q | 4 - .../orderby_position_unsupported.q | 4 - .../queries/clientnegative/orderbysortby.q | 8 -- .../queries/clientnegative/parquet_char.q | 3 - .../queries/clientnegative/parquet_date.q | 3 - .../queries/clientnegative/parquet_decimal.q | 3 - .../clientnegative/parquet_timestamp.q | 3 - .../queries/clientnegative/parquet_varchar.q | 3 - .../clientnegative/part_col_complex_type.q | 1 - .../queries/clientnegative/protectmode_part.q | 15 ---- .../clientnegative/protectmode_part1.q | 21 ------ .../clientnegative/protectmode_part2.q | 9 --- .../clientnegative/protectmode_part_no_drop.q | 10 --- .../queries/clientnegative/protectmode_tbl1.q | 8 -- .../queries/clientnegative/protectmode_tbl2.q | 12 --- .../queries/clientnegative/protectmode_tbl3.q | 10 --- .../queries/clientnegative/protectmode_tbl4.q | 15 ---- .../queries/clientnegative/protectmode_tbl5.q | 15 ---- .../queries/clientnegative/protectmode_tbl6.q | 8 -- .../queries/clientnegative/protectmode_tbl7.q | 13 ---- .../queries/clientnegative/protectmode_tbl8.q | 13 ---- .../clientnegative/protectmode_tbl_no_drop.q | 9 --- ...ptf_negative_AggrFuncsWithNoGBYNoPartDef.q | 20 ----- .../ptf_negative_AmbiguousWindowDefn.q | 28 ------- .../ptf_negative_DistributeByOrderBy.q | 19 ----- .../ptf_negative_DuplicateWindowAlias.q | 22 ------ ..._negative_HavingLeadWithNoGBYNoWindowing.q | 20 ----- .../ptf_negative_HavingLeadWithPTF.q | 22 ------ .../ptf_negative_InvalidValueBoundary.q | 21 ------ .../ptf_negative_JoinWithAmbigousAlias.q | 20 ----- .../ptf_negative_PartitionBySortBy.q | 19 ----- .../ptf_negative_WhereWithRankCond.q | 21 ------ .../clientnegative/ptf_window_boundaries.q | 17 ----- .../clientnegative/ptf_window_boundaries2.q | 17 ----- .../queries/clientnegative/recursive_view.q | 15 ---- .../test/queries/clientnegative/regex_col_1.q | 3 - .../test/queries/clientnegative/regex_col_2.q | 3 - .../clientnegative/regex_col_groupby.q | 3 - .../queries/clientnegative/sa_fail_hook3.q | 4 - .../src/test/queries/clientnegative/sample.q | 1 - .../clientnegative/script_broken_pipe2.q | 3 - .../clientnegative/script_broken_pipe3.q | 3 - .../queries/clientnegative/script_error.q | 7 -- .../clientnegative/select_charliteral.q | 3 - .../clientnegative/select_udtf_alias.q | 3 - .../test/queries/clientnegative/semijoin1.q | 2 - .../test/queries/clientnegative/semijoin2.q | 2 - .../test/queries/clientnegative/semijoin3.q | 2 - .../test/queries/clientnegative/semijoin4.q | 3 - .../test/queries/clientnegative/serde_regex.q | 17 ----- .../queries/clientnegative/serde_regex2.q | 23 ------ .../queries/clientnegative/serde_regex3.q | 14 ---- .../clientnegative/set_hiveconf_validation0.q | 5 -- .../clientnegative/set_hiveconf_validation1.q | 5 -- .../clientnegative/set_hiveconf_validation2.q | 5 -- .../clientnegative/set_table_property.q | 4 - .../queries/clientnegative/show_columns1.q | 2 - .../queries/clientnegative/show_columns2.q | 2 - .../queries/clientnegative/show_columns3.q | 7 -- .../show_create_table_does_not_exist.q | 2 - .../clientnegative/show_create_table_index.q | 6 -- .../queries/clientnegative/show_partitions1.q | 1 - .../clientnegative/show_tableproperties1.q | 1 - .../queries/clientnegative/show_tables_bad1.q | 1 - .../queries/clientnegative/show_tables_bad2.q | 1 - .../clientnegative/show_tables_bad_db1.q | 1 - .../clientnegative/show_tables_bad_db2.q | 1 - .../queries/clientnegative/show_tablestatus.q | 1 - .../show_tablestatus_not_existing_part.q | 1 - .../clientnegative/smb_bucketmapjoin.q | 23 ------ .../queries/clientnegative/smb_mapjoin_14.q | 38 ---------- .../sortmerge_mapjoin_mismatch_1.q | 28 ------- .../split_sample_out_of_range.q | 3 - .../split_sample_wrong_format.q | 3 - .../split_sample_wrong_format2.q | 3 - .../clientnegative/stats_aggregator_error_1.q | 18 ----- .../clientnegative/stats_aggregator_error_2.q | 16 ---- .../clientnegative/stats_noscan_non_native.q | 6 -- .../stats_partialscan_autogether.q | 31 -------- .../stats_partialscan_non_external.q | 5 -- .../stats_partialscan_non_native.q | 6 -- .../clientnegative/stats_partscan_norcfile.q | 12 --- .../clientnegative/stats_publisher_error_1.q | 18 ----- .../clientnegative/stats_publisher_error_2.q | 16 ---- .../test/queries/clientnegative/strict_join.q | 3 - .../queries/clientnegative/strict_orderby.q | 7 -- .../queries/clientnegative/strict_pruning.q | 6 -- .../test/queries/clientnegative/subq_insert.q | 2 - .../subquery_exists_implicit_gby.q | 10 --- .../clientnegative/subquery_in_groupby.q | 5 -- .../clientnegative/subquery_in_select.q | 6 -- .../subquery_multiple_cols_in_select.q | 7 -- .../clientnegative/subquery_nested_subquery.q | 18 ----- .../subquery_notexists_implicit_gby.q | 10 --- .../clientnegative/subquery_shared_alias.q | 6 -- .../clientnegative/subquery_subquery_chain.q | 6 -- .../subquery_unqual_corr_expr.q | 6 -- .../clientnegative/subquery_windowing_corr.q | 26 ------- .../clientnegative/subquery_with_or_cond.q | 5 -- .../src/test/queries/clientnegative/touch1.q | 1 - .../src/test/queries/clientnegative/touch2.q | 1 - .../clientnegative/truncate_bucketed_column.q | 7 -- .../truncate_column_indexed_table.q | 9 --- .../truncate_column_list_bucketing.q | 14 ---- .../clientnegative/truncate_column_seqfile.q | 7 -- .../truncate_nonexistant_column.q | 7 -- .../truncate_partition_column.q | 7 -- .../truncate_partition_column2.q | 7 -- .../clientnegative/truncate_table_failure1.q | 2 - .../clientnegative/truncate_table_failure2.q | 2 - .../clientnegative/truncate_table_failure3.q | 4 - .../clientnegative/truncate_table_failure4.q | 5 -- .../clientnegative/udaf_invalid_place.q | 1 - .../udf_array_contains_wrong1.q | 2 - .../udf_array_contains_wrong2.q | 2 - .../queries/clientnegative/udf_assert_true.q | 7 -- .../queries/clientnegative/udf_assert_true2.q | 2 - .../queries/clientnegative/udf_coalesce.q | 1 - .../clientnegative/udf_concat_ws_wrong1.q | 2 - .../clientnegative/udf_concat_ws_wrong2.q | 2 - .../clientnegative/udf_concat_ws_wrong3.q | 2 - .../clientnegative/udf_elt_wrong_args_len.q | 1 - .../clientnegative/udf_elt_wrong_type.q | 3 - .../clientnegative/udf_field_wrong_args_len.q | 1 - .../clientnegative/udf_field_wrong_type.q | 3 - .../clientnegative/udf_format_number_wrong1.q | 2 - .../clientnegative/udf_format_number_wrong2.q | 2 - .../clientnegative/udf_format_number_wrong3.q | 2 - .../clientnegative/udf_format_number_wrong4.q | 2 - .../clientnegative/udf_format_number_wrong5.q | 2 - .../clientnegative/udf_format_number_wrong6.q | 2 - .../clientnegative/udf_format_number_wrong7.q | 2 - .../udf_function_does_not_implement_udf.q | 1 - .../queries/clientnegative/udf_if_not_bool.q | 1 - .../clientnegative/udf_if_wrong_args_len.q | 1 - .../src/test/queries/clientnegative/udf_in.q | 1 - .../clientnegative/udf_instr_wrong_args_len.q | 1 - .../clientnegative/udf_instr_wrong_type.q | 3 - .../test/queries/clientnegative/udf_invalid.q | 1 - .../clientnegative/udf_local_resource.q | 1 - .../udf_locate_wrong_args_len.q | 1 - .../clientnegative/udf_locate_wrong_type.q | 3 - .../clientnegative/udf_map_keys_arg_num.q | 1 - .../clientnegative/udf_map_keys_arg_type.q | 1 - .../clientnegative/udf_map_values_arg_num.q | 1 - .../clientnegative/udf_map_values_arg_type.q | 1 - .../src/test/queries/clientnegative/udf_max.q | 2 - .../src/test/queries/clientnegative/udf_min.q | 2 - .../clientnegative/udf_nonexistent_resource.q | 1 - .../clientnegative/udf_printf_wrong1.q | 2 - .../clientnegative/udf_printf_wrong2.q | 2 - .../clientnegative/udf_printf_wrong3.q | 2 - .../clientnegative/udf_printf_wrong4.q | 2 - .../clientnegative/udf_qualified_name.q | 1 - .../queries/clientnegative/udf_reflect_neg.q | 9 --- .../clientnegative/udf_size_wrong_args_len.q | 5 -- .../clientnegative/udf_size_wrong_type.q | 1 - .../clientnegative/udf_sort_array_wrong1.q | 2 - .../clientnegative/udf_sort_array_wrong2.q | 2 - .../clientnegative/udf_sort_array_wrong3.q | 2 - .../queries/clientnegative/udf_test_error.q | 3 - .../clientnegative/udf_test_error_reduce.q | 11 --- .../clientnegative/udf_when_type_wrong.q | 6 -- .../udtf_explode_not_supported1.q | 1 - .../udtf_explode_not_supported2.q | 1 - .../udtf_explode_not_supported3.q | 1 - .../udtf_explode_not_supported4.q | 1 - .../clientnegative/udtf_invalid_place.q | 1 - .../clientnegative/udtf_not_supported1.q | 1 - .../clientnegative/udtf_not_supported3.q | 1 - .../src/test/queries/clientnegative/union2.q | 13 ---- .../src/test/queries/clientnegative/union22.q | 26 ------- .../src/test/queries/clientnegative/union3.q | 5 -- .../test/queries/clientnegative/uniquejoin.q | 3 - .../test/queries/clientnegative/uniquejoin2.q | 3 - .../test/queries/clientnegative/uniquejoin3.q | 3 - .../clientnegative/unset_table_property.q | 6 -- .../clientnegative/unset_view_property.q | 6 -- .../clientnegative/windowing_invalid_udaf.q | 1 - .../windowing_leadlag_in_udaf.q | 15 ---- .../clientnegative/windowing_ll_no_neg.q | 26 ------- .../clientnegative/windowing_ll_no_over.q | 17 ----- .../clientnegative/wrong_column_type.q | 4 - .../queries/negative/ambiguous_join_col.q | 2 - .../test/queries/negative/duplicate_alias.q | 2 - .../ql/src/test/queries/negative/garbage.q | 1 - .../negative/insert_wrong_number_columns.q | 2 - .../queries/negative/invalid_create_table.q | 4 - .../src/test/queries/negative/invalid_dot.q | 2 - .../negative/invalid_function_param2.q | 2 - .../src/test/queries/negative/invalid_index.q | 2 - .../queries/negative/invalid_list_index.q | 2 - .../queries/negative/invalid_list_index2.q | 2 - .../test/queries/negative/invalid_map_index.q | 2 - .../queries/negative/invalid_map_index2.q | 2 - .../test/queries/negative/invalid_select.q | 4 - .../queries/negative/macro_reserved_word.q | 1 - .../test/queries/negative/missing_overwrite.q | 2 - .../test/queries/negative/nonkey_groupby.q | 2 - .../src/test/queries/negative/quoted_string.q | 2 - .../test/queries/negative/unknown_column1.q | 2 - .../test/queries/negative/unknown_column2.q | 2 - .../test/queries/negative/unknown_column3.q | 2 - .../test/queries/negative/unknown_column4.q | 2 - .../test/queries/negative/unknown_column5.q | 2 - .../test/queries/negative/unknown_column6.q | 2 - .../test/queries/negative/unknown_function1.q | 2 - .../test/queries/negative/unknown_function2.q | 2 - .../test/queries/negative/unknown_function3.q | 2 - .../test/queries/negative/unknown_function4.q | 2 - .../test/queries/negative/unknown_table1.q | 2 - .../test/queries/negative/unknown_table2.q | 2 - .../test/queries/negative/wrong_distinct1.q | 2 - .../test/queries/negative/wrong_distinct2.q | 2 - .../test/queries/positive/case_sensitivity.q | 2 - .../ql/src/test/queries/positive/cast1.q | 2 - .../ql/src/test/queries/positive/groupby1.q | 2 - .../ql/src/test/queries/positive/groupby2.q | 2 - .../ql/src/test/queries/positive/groupby3.q | 2 - .../ql/src/test/queries/positive/groupby4.q | 2 - .../ql/src/test/queries/positive/groupby5.q | 4 - .../ql/src/test/queries/positive/groupby6.q | 2 - .../ql/src/test/queries/positive/input1.q | 2 - .../ql/src/test/queries/positive/input2.q | 4 - .../ql/src/test/queries/positive/input20.q | 9 --- .../ql/src/test/queries/positive/input3.q | 5 -- .../ql/src/test/queries/positive/input4.q | 7 -- .../ql/src/test/queries/positive/input5.q | 7 -- .../ql/src/test/queries/positive/input6.q | 2 - .../ql/src/test/queries/positive/input7.q | 2 - .../ql/src/test/queries/positive/input8.q | 2 - .../ql/src/test/queries/positive/input9.q | 2 - .../src/test/queries/positive/input_part1.q | 2 - .../queries/positive/input_testsequencefile.q | 2 - .../test/queries/positive/input_testxpath.q | 2 - .../test/queries/positive/input_testxpath2.q | 2 - .../ql/src/test/queries/positive/join1.q | 3 - .../ql/src/test/queries/positive/join2.q | 3 - .../ql/src/test/queries/positive/join3.q | 4 - .../ql/src/test/queries/positive/join4.q | 14 ---- .../ql/src/test/queries/positive/join5.q | 15 ---- .../ql/src/test/queries/positive/join6.q | 16 ---- .../ql/src/test/queries/positive/join7.q | 21 ------ .../ql/src/test/queries/positive/join8.q | 14 ---- .../ql/src/test/queries/positive/sample1.q | 5 -- .../ql/src/test/queries/positive/sample2.q | 4 - .../ql/src/test/queries/positive/sample3.q | 4 - .../ql/src/test/queries/positive/sample4.q | 4 - .../ql/src/test/queries/positive/sample5.q | 3 - .../ql/src/test/queries/positive/sample6.q | 3 - .../ql/src/test/queries/positive/sample7.q | 4 - .../ql/src/test/queries/positive/subq.q | 4 - .../ql/src/test/queries/positive/udf1.q | 5 -- .../ql/src/test/queries/positive/udf4.q | 1 - .../ql/src/test/queries/positive/udf6.q | 1 - .../ql/src/test/queries/positive/udf_case.q | 10 --- .../ql/src/test/queries/positive/udf_when.q | 10 --- .../ql/src/test/queries/positive/union.q | 6 -- 690 files changed, 5352 deletions(-) delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_00.qv delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_01.qv delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/add_partition_with_whitelist.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/addpart1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_non_native.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_2columns.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidcolname.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidtype.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_offline.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_with_whitelist.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_table_add_partition.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_table_wrong_regex.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_as_select_not_exist.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_as_select_with_partition.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure5.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure6.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure7.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure8.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure9.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/altern1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col0.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_non_existent_tbl.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_view.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive5.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_corrupt.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi5.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi6.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi7.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec5.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addjar.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addpartition.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner_default.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_all_role.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_default_role.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_none_role.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_caseinsensitivity.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_macro1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_role_no_admin.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_createview.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_ctas.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_desc_table_nosel.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_dfs.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_disallow_transform.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_cascade.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_empty.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_role_no_admin.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_droppartition.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_5.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_6.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_7.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_create_db.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_drop_db.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_dup.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noinspriv.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noselectpriv.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_rename.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_serdeprop.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_tab.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_view.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_part.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_priv_current_role_neg.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_create.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_drop.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_grant.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_rolehierarchy_privs.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select_view.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_parts_nosel.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_no_admin.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_v1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_roles_no_admin.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_truncate.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_add_partition.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_alterpart_loc.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_altertab_setloc.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table_ext.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_createdb.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_export.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_import.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_index.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert_local.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_load_data.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_create_tbl.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_grant_public.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_revoke_public.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/autolocal1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_exec_hooks.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_indextype.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_sample_clause.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cachingprintstream.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cluster_tasklog_retrieval.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbydistributeby.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbyorderby.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbysortby.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_change_skewedcol_type1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename5.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compare_double_bigint.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compare_string_bigint.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compile_processor.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compute_stats_long.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_class.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_db.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonudf_class.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_insert_outputformat.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view5.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view6.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view7.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view8.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_col_name_value_no_mismatch.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_dup_col_name.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_failure_invalid_col_name.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_wrong_regex.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_udaf_failure.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_unknown_genericudf.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_unknown_udf_udaf.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure5.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure6.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure7.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure8.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure9.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ctas.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_recursion.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_with_in_subquery.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_create_already_exists.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_create_invalid_name.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_not_empty.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_not_empty_restrict.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodblock.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodbunlock.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notablelock.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notableunlock.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ddltime.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/decimal_precision.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/decimal_precision_1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/default_partition_name.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/deletejar.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_func_nonexistent.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_function_failure.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_index_failure.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_native_udf.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_failure.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_filter_failure.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_view_failure1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_view_failure2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_alias_in_transform.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_alias_in_transform_schema.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_empty.q.disabled delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_max.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_max_per_node.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_test.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_12_nonnative_export.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_22_export_authfail.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/external1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/external2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fetchtask_ioexception.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/file_with_header_footer_negative.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_bad_class.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_void_input.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_void_output.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fs_default_name1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fs_default_name2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/genericFileFormat.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby2_map_skew_multi_distinct.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby2_multi_distinct.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby3_map_skew_multi_distinct.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby3_multi_distinct.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_cube1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_cube2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_id1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets5.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets6.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets7.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_invalid_position.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_key.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_rollup1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_rollup2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/having1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_bitmap_no_map_aggr.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_entry_limit.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_size_limit.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input41.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input_part0_neg.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into5.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into6.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_view_failure.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertexternal1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_arithmetic_type.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_avg_syntax.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_5.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_6.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_5.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_6.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_config1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_config2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_create_tbl1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_create_tbl2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_mapjoin1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_max_syntax.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_min_syntax.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column_with_subquery.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column_with_tablename.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_expression.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_std_syntax.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_sum_syntax.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_alter1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_alter2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_create2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_transform.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_tbl_name.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_variance_syntax.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalidate_view1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join28.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join29.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join32.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join35.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_alt_syntax_comma_on.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous_vc.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_nonexistent_part.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/joinneg.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lateral_view_alias.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lateral_view_join.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition_stats.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/line_terminator.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_exist_part_authfail.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_non_native.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_nonpart_authfail.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_authfail.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_nospec.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_stored_as_dirs.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_view_failure.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_noof_part.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/local_mapred_error_cache.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg5.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_query_tbl_in_locked_db.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_db_lock_conflict.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_drop_locked_db.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_lock_db_in_use.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/macro_unused_parameter.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_hadoop20.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/minimr_broken_pipe.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nested_complex_neg.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/no_matching_udf.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nonkey_groupby.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_insert.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_load.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/notable_alias4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderby_invalid_position.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderby_position_unsupported.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderbysortby.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_char.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_date.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_decimal.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_timestamp.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_varchar.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/part_col_complex_type.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl5.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl6.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl7.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl8.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_DistributeByOrderBy.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_DuplicateWindowAlias.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithPTF.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_InvalidValueBoundary.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_JoinWithAmbigousAlias.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_PartitionBySortBy.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_WhereWithRankCond.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_window_boundaries.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_window_boundaries2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/recursive_view.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_groupby.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sa_fail_hook3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sample.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_error.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/select_charliteral.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/select_udtf_alias.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation0.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_table_property.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_create_table_does_not_exist.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_create_table_index.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_partitions1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tableproperties1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad_db1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad_db2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tablestatus.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tablestatus_not_existing_part.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/smb_bucketmapjoin.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/smb_mapjoin_14.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sortmerge_mapjoin_mismatch_1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_out_of_range.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_wrong_format.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_wrong_format2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_noscan_non_native.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_autogether.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_non_external.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_non_native.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partscan_norcfile.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_join.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_orderby.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_pruning.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subq_insert.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_exists_implicit_gby.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_groupby.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_select.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_multiple_cols_in_select.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_nested_subquery.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_notexists_implicit_gby.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_shared_alias.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_subquery_chain.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_unqual_corr_expr.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_windowing_corr.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_with_or_cond.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/touch1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/touch2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_bucketed_column.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_indexed_table.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_list_bucketing.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_seqfile.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_nonexistant_column.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_partition_column.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_partition_column2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udaf_invalid_place.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_array_contains_wrong1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_array_contains_wrong2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_assert_true.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_assert_true2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_coalesce.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_elt_wrong_args_len.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_elt_wrong_type.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_field_wrong_args_len.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_field_wrong_type.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong5.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong6.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong7.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_function_does_not_implement_udf.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_if_not_bool.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_if_wrong_args_len.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_in.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_instr_wrong_args_len.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_instr_wrong_type.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_invalid.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_local_resource.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_locate_wrong_args_len.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_locate_wrong_type.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_keys_arg_num.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_keys_arg_type.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_values_arg_num.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_values_arg_type.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_max.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_min.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_nonexistent_resource.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_qualified_name.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_reflect_neg.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_size_wrong_args_len.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_size_wrong_type.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_test_error.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_test_error_reduce.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_invalid_place.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_not_supported1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_not_supported3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union22.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/unset_table_property.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/unset_view_property.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_invalid_udaf.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_leadlag_in_udaf.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_neg.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_over.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/wrong_column_type.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/ambiguous_join_col.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/duplicate_alias.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/garbage.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/insert_wrong_number_columns.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_create_table.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_dot.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_function_param2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_index.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_list_index.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_list_index2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_map_index.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_map_index2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_select.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/macro_reserved_word.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/missing_overwrite.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/nonkey_groupby.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/quoted_string.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column5.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column6.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_table1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_table2.q delete mode 100755 sql/hive/src/test/resources/ql/src/test/queries/negative/wrong_distinct1.q delete mode 100755 sql/hive/src/test/resources/ql/src/test/queries/negative/wrong_distinct2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/case_sensitivity.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/cast1.q delete mode 100755 sql/hive/src/test/resources/ql/src/test/queries/positive/groupby1.q delete mode 100755 sql/hive/src/test/resources/ql/src/test/queries/positive/groupby2.q delete mode 100755 sql/hive/src/test/resources/ql/src/test/queries/positive/groupby3.q delete mode 100755 sql/hive/src/test/resources/ql/src/test/queries/positive/groupby4.q delete mode 100755 sql/hive/src/test/resources/ql/src/test/queries/positive/groupby5.q delete mode 100755 sql/hive/src/test/resources/ql/src/test/queries/positive/groupby6.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/input1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/input2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/input20.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/input3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/input4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/input5.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/input6.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/input7.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/input8.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/input9.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/input_part1.q delete mode 100755 sql/hive/src/test/resources/ql/src/test/queries/positive/input_testsequencefile.q delete mode 100755 sql/hive/src/test/resources/ql/src/test/queries/positive/input_testxpath.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/input_testxpath2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/join1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/join2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/join3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/join4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/join5.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/join6.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/join7.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/join8.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/sample1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/sample2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/sample3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/sample4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/sample5.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/sample6.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/sample7.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/subq.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/udf1.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/udf4.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/udf6.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/udf_case.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/udf_when.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/union.q diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs.q deleted file mode 100644 index c640ca148b70b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs.q +++ /dev/null @@ -1,43 +0,0 @@ - -select - cdouble - ,Round(cdouble, 2) - ,Floor(cdouble) - ,Ceil(cdouble) - ,Rand(98007) as rnd - ,Exp(ln(cdouble)) - ,Ln(cdouble) - ,Ln(cfloat) - ,Log10(cdouble) - -- Use log2 as a representative function to test all input types. - ,Log2(cdouble) - ,Log2(cfloat) - ,Log2(cbigint) - ,Log2(cint) - ,Log2(csmallint) - ,Log2(ctinyint) - ,Log(2.0, cdouble) - ,Pow(log2(cdouble), 2.0) - ,Power(log2(cdouble), 2.0) - ,Sqrt(cdouble) - ,Sqrt(cbigint) - ,Bin(cbigint) - ,Hex(cdouble) - ,Conv(cbigint, 10, 16) - ,Abs(cdouble) - ,Abs(ctinyint) - ,Pmod(cint, 3) - ,Sin(cdouble) - ,Asin(cdouble) - ,Cos(cdouble) - ,ACos(cdouble) - ,Atan(cdouble) - ,Degrees(cdouble) - ,Radians(cdouble) - ,Positive(cdouble) - ,Positive(cbigint) - ,Negative(cdouble) - ,Sign(cdouble) - ,Sign(cbigint) -from alltypesorc order by rnd limit 400; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_00.qv b/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_00.qv deleted file mode 100644 index 51f231008f6d2..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_00.qv +++ /dev/null @@ -1 +0,0 @@ -SET hive.vectorized.execution.enabled = false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_01.qv b/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_01.qv deleted file mode 100644 index 18e02dc854baf..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_01.qv +++ /dev/null @@ -1 +0,0 @@ -SET hive.vectorized.execution.enabled = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/add_partition_with_whitelist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/add_partition_with_whitelist.q deleted file mode 100644 index 8f0a60b713ab8..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/add_partition_with_whitelist.q +++ /dev/null @@ -1,8 +0,0 @@ -SET hive.metastore.partition.name.whitelist.pattern=[\\x20-\\x7E&&[^,]]* ; --- This pattern matches all printable ASCII characters (disallow unicode) and disallows commas - -CREATE TABLE part_whitelist_test (key STRING, value STRING) PARTITIONED BY (ds STRING); -SHOW PARTITIONS part_whitelist_test; - -ALTER TABLE part_whitelist_test ADD PARTITION (ds='1,2,3,4'); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/addpart1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/addpart1.q deleted file mode 100644 index a7c9fe91f6cd9..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/addpart1.q +++ /dev/null @@ -1,11 +0,0 @@ - -create table addpart1 (a int) partitioned by (b string, c string); - -alter table addpart1 add partition (b='f', c='s'); - -show partitions addpart1; - -alter table addpart1 add partition (b='f', c=''); - -show prtitions addpart1; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q deleted file mode 100644 index 4193315d30043..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q +++ /dev/null @@ -1,16 +0,0 @@ -set hive.exec.concatenate.check.index=true; -create table src_rc_concatenate_test(key int, value string) stored as rcfile; - -load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_concatenate_test; -load data local inpath '../../data/files/smbbucket_2.rc' into table src_rc_concatenate_test; -load data local inpath '../../data/files/smbbucket_3.rc' into table src_rc_concatenate_test; - -show table extended like `src_rc_concatenate_test`; - -select count(1) from src_rc_concatenate_test; -select sum(hash(key)), sum(hash(value)) from src_rc_concatenate_test; - -create index src_rc_concatenate_test_index on table src_rc_concatenate_test(key) as 'compact' WITH DEFERRED REBUILD IDXPROPERTIES ("prop1"="val1", "prop2"="val2"); -show indexes on src_rc_concatenate_test; - -alter table src_rc_concatenate_test concatenate; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_non_native.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_non_native.q deleted file mode 100644 index 73ae853778832..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_non_native.q +++ /dev/null @@ -1,6 +0,0 @@ - -CREATE TABLE non_native1(key int, value string) -STORED BY 'org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler'; - --- we do not support ALTER TABLE on non-native tables yet -ALTER TABLE non_native1 RENAME TO new_non_native; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_2columns.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_2columns.q deleted file mode 100644 index e10f77cf3f161..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_2columns.q +++ /dev/null @@ -1,11 +0,0 @@ --- create testing table -create table alter_coltype(key string, value string) partitioned by (dt string, ts string); - --- insert and create a partition -insert overwrite table alter_coltype partition(dt='100x', ts='6:30pm') select * from src1; - -desc alter_coltype; - --- alter partition change multiple keys at same time -alter table alter_coltype partition column (dt int, ts int); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidcolname.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidcolname.q deleted file mode 100644 index 66eba75d4084d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidcolname.q +++ /dev/null @@ -1,12 +0,0 @@ --- create testing table -create table alter_coltype(key string, value string) partitioned by (dt string, ts string); - --- insert and create a partition -insert overwrite table alter_coltype partition(dt='100x', ts='6:30pm') select * from src1; - -desc alter_coltype; - --- alter partition key column with invalid column name -alter table alter_coltype partition column (dd int); - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidtype.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidtype.q deleted file mode 100644 index ad016c5f3a76e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidtype.q +++ /dev/null @@ -1,11 +0,0 @@ --- create testing table -create table alter_coltype(key string, value string) partitioned by (dt string, ts string); - --- insert and create a partition -insert overwrite table alter_coltype partition(dt='100x', ts='6:30pm') select * from src1; - -desc alter_coltype; - --- alter partition key column data type for ts column to a wrong type -alter table alter_coltype partition column (ts time); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q deleted file mode 100644 index 8cbb25cfa9725..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q +++ /dev/null @@ -1,8 +0,0 @@ --- Create table -create table if not exists alter_part_invalidspec(key string, value string ) partitioned by (year string, month string) stored as textfile ; - --- Load data -load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_invalidspec partition (year='1996', month='10'); -load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_invalidspec partition (year='1996', month='12'); - -alter table alter_part_invalidspec partition (year='1997') enable no_drop; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop.q deleted file mode 100644 index 3c0ff02b1ac1d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop.q +++ /dev/null @@ -1,9 +0,0 @@ --- Create table -create table if not exists alter_part_nodrop_part(key string, value string ) partitioned by (year string, month string) stored as textfile ; - --- Load data -load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_nodrop_part partition (year='1996', month='10'); -load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_nodrop_part partition (year='1996', month='12'); - -alter table alter_part_nodrop_part partition (year='1996') enable no_drop; -alter table alter_part_nodrop_part drop partition (year='1996'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q deleted file mode 100644 index f2135b1aa02e4..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q +++ /dev/null @@ -1,9 +0,0 @@ --- Create table -create table if not exists alter_part_nodrop_table(key string, value string ) partitioned by (year string, month string) stored as textfile ; - --- Load data -load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_nodrop_table partition (year='1996', month='10'); -load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_nodrop_table partition (year='1996', month='12'); - -alter table alter_part_nodrop_table partition (year='1996') enable no_drop; -drop table alter_part_nodrop_table; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_offline.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_offline.q deleted file mode 100644 index 7376d8bfe4a74..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_offline.q +++ /dev/null @@ -1,11 +0,0 @@ --- create table -create table if not exists alter_part_offline (key string, value string ) partitioned by (year string, month string) stored as textfile ; - --- Load data -load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_offline partition (year='1996', month='10'); -load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_offline partition (year='1996', month='12'); - -alter table alter_part_offline partition (year='1996') disable offline; -select * from alter_part_offline where year = '1996'; -alter table alter_part_offline partition (year='1996') enable offline; -select * from alter_part_offline where year = '1996'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_with_whitelist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_with_whitelist.q deleted file mode 100644 index 6e33bc0782d29..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_with_whitelist.q +++ /dev/null @@ -1,9 +0,0 @@ -SET hive.metastore.partition.name.whitelist.pattern=[\\x20-\\x7E&&[^,]]* ; --- This pattern matches all printable ASCII characters (disallow unicode) and disallows commas - -CREATE TABLE part_whitelist_test (key STRING, value STRING) PARTITIONED BY (ds STRING); -SHOW PARTITIONS part_whitelist_test; - -ALTER TABLE part_whitelist_test ADD PARTITION (ds='1'); - -ALTER TABLE part_whitelist_test PARTITION (ds='1') rename to partition (ds='1,2,3'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q deleted file mode 100644 index be971f1849869..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q +++ /dev/null @@ -1,6 +0,0 @@ -create table alter_rename_partition_src ( col1 string ) stored as textfile ; -load data local inpath '../../data/files/test.dat' overwrite into table alter_rename_partition_src ; -create table alter_rename_partition ( col1 string ) partitioned by (pcol1 string , pcol2 string) stored as sequencefile; -insert overwrite table alter_rename_partition partition (pCol1='old_part1:', pcol2='old_part2:') select col1 from alter_rename_partition_src ; - -alter table alter_rename_partition partition (pCol1='nonexist_part1:', pcol2='nonexist_part2:') rename to partition (pCol1='new_part1:', pcol2='new_part2:'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q deleted file mode 100644 index 4babdda2dbe2a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q +++ /dev/null @@ -1,6 +0,0 @@ -create table alter_rename_partition_src ( col1 string ) stored as textfile ; -load data local inpath '../../data/files/test.dat' overwrite into table alter_rename_partition_src ; -create table alter_rename_partition ( col1 string ) partitioned by (pcol1 string , pcol2 string) stored as sequencefile; -insert overwrite table alter_rename_partition partition (pCol1='old_part1:', pcol2='old_part2:') select col1 from alter_rename_partition_src ; - -alter table alter_rename_partition partition (pCol1='old_part1:', pcol2='old_part2:') rename to partition (pCol1='old_part1:', pcol2='old_part2:'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q deleted file mode 100644 index 3af807ef61217..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q +++ /dev/null @@ -1,6 +0,0 @@ -create table alter_rename_partition_src ( col1 string ) stored as textfile ; -load data local inpath '../../data/files/test.dat' overwrite into table alter_rename_partition_src ; -create table alter_rename_partition ( col1 string ) partitioned by (pcol1 string , pcol2 string) stored as sequencefile; -insert overwrite table alter_rename_partition partition (pCol1='old_part1:', pcol2='old_part2:') select col1 from alter_rename_partition_src ; - -alter table alter_rename_partition partition (pCol1='old_part1:', pcol2='old_part2:') rename to partition (pCol1='old_part1:', pcol2='old_part2:', pcol3='old_part3:'); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_table_add_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_table_add_partition.q deleted file mode 100644 index 2427c3b2a45f8..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_table_add_partition.q +++ /dev/null @@ -1,5 +0,0 @@ -create table mp (a int) partitioned by (b int); - --- should fail -alter table mp add partition (b='1', c='1'); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_table_wrong_regex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_table_wrong_regex.q deleted file mode 100644 index fad194d016ec9..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_table_wrong_regex.q +++ /dev/null @@ -1,7 +0,0 @@ -drop table aa; -create table aa ( test STRING ) - ROW FORMAT SERDE 'org.apache.hadoop.hive.contrib.serde2.RegexSerDe' - WITH SERDEPROPERTIES ("input.regex" = "(.*)", "output.format.string" = "$1s"); - -alter table aa set serdeproperties ("input.regex" = "[^\\](.*)", "output.format.string" = "$1s"); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_as_select_not_exist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_as_select_not_exist.q deleted file mode 100644 index 30fe4d9916ab3..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_as_select_not_exist.q +++ /dev/null @@ -1,4 +0,0 @@ -DROP VIEW testView; - --- Cannot ALTER VIEW AS SELECT if view currently does not exist -ALTER VIEW testView AS SELECT * FROM srcpart; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_as_select_with_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_as_select_with_partition.q deleted file mode 100644 index dca6770b1b17e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_as_select_with_partition.q +++ /dev/null @@ -1,12 +0,0 @@ -CREATE VIEW testViewPart PARTITIONED ON (value) -AS -SELECT key, value -FROM src -WHERE key=86; - -ALTER VIEW testViewPart -ADD PARTITION (value='val_86') PARTITION (value='val_xyz'); -DESCRIBE FORMATTED testViewPart; - --- If a view has partition, could not replace it with ALTER VIEW AS SELECT -ALTER VIEW testViewPart as SELECT * FROM srcpart; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure.q deleted file mode 100644 index 705b985095fa3..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure.q +++ /dev/null @@ -1,3 +0,0 @@ -DROP VIEW xxx3; -CREATE VIEW xxx3 AS SELECT * FROM src; -ALTER TABLE xxx3 REPLACE COLUMNS (xyz int); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure2.q deleted file mode 100644 index 26d2c4f3ad2fb..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure2.q +++ /dev/null @@ -1,8 +0,0 @@ -DROP VIEW xxx4; -CREATE VIEW xxx4 -PARTITIONED ON (value) -AS -SELECT * FROM src; - --- should fail: need to use ALTER VIEW, not ALTER TABLE -ALTER TABLE xxx4 ADD PARTITION (value='val_86'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure3.q deleted file mode 100644 index 49c17a8b573c9..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure3.q +++ /dev/null @@ -1,2 +0,0 @@ --- should fail: can't use ALTER VIEW on a table -ALTER VIEW srcpart ADD PARTITION (ds='2012-12-31', hr='23'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure4.q deleted file mode 100644 index e2fad270b1d89..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure4.q +++ /dev/null @@ -1,8 +0,0 @@ -DROP VIEW xxx5; -CREATE VIEW xxx5 -PARTITIONED ON (value) -AS -SELECT * FROM src; - --- should fail: LOCATION clause is illegal -ALTER VIEW xxx5 ADD PARTITION (value='val_86') LOCATION '/foo/bar/baz'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure5.q deleted file mode 100644 index e44766e113063..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure5.q +++ /dev/null @@ -1,8 +0,0 @@ -DROP VIEW xxx6; -CREATE VIEW xxx6 -PARTITIONED ON (value) -AS -SELECT * FROM src; - --- should fail: partition column name does not match -ALTER VIEW xxx6 ADD PARTITION (v='val_86'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure6.q deleted file mode 100644 index dab7b145f7c4e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure6.q +++ /dev/null @@ -1,11 +0,0 @@ -DROP VIEW xxx7; -CREATE VIEW xxx7 -PARTITIONED ON (key) -AS -SELECT hr,key FROM srcpart; - -SET hive.mapred.mode=strict; - --- strict mode should cause this to fail since view partition --- predicate does not correspond to an underlying table partition predicate -ALTER VIEW xxx7 ADD PARTITION (key=10); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure7.q deleted file mode 100644 index eff04c5b47dee..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure7.q +++ /dev/null @@ -1,8 +0,0 @@ -DROP VIEW xxx8; -CREATE VIEW xxx8 -PARTITIONED ON (ds,hr) -AS -SELECT key,ds,hr FROM srcpart; - --- should fail: need to fill in all partition columns -ALTER VIEW xxx8 ADD PARTITION (ds='2011-01-01'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure8.q deleted file mode 100644 index 9dff784250616..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure8.q +++ /dev/null @@ -1,3 +0,0 @@ --- should fail: can't use ALTER VIEW on a table -CREATE TABLE invites (foo INT, bar STRING) PARTITIONED BY (ds STRING); -ALTER VIEW invites RENAME TO invites2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure9.q deleted file mode 100644 index 0f40fad90d974..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure9.q +++ /dev/null @@ -1,7 +0,0 @@ -DROP VIEW xxx4; -CREATE VIEW xxx4 -AS -SELECT * FROM src; - --- should fail: need to use ALTER VIEW, not ALTER TABLE -ALTER TABLE xxx4 RENAME TO xxx4a; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/altern1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/altern1.q deleted file mode 100644 index 60414c1f3a7a5..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/altern1.q +++ /dev/null @@ -1,4 +0,0 @@ - -create table altern1(a int, b int) partitioned by (ds string); -alter table altern1 replace columns(a int, b int, ds string); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col.q deleted file mode 100644 index 866cec126f782..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col.q +++ /dev/null @@ -1 +0,0 @@ -FROM (SELECT key, concat(value) AS key FROM src) a SELECT a.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col0.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col0.q deleted file mode 100644 index 46349c60bc79b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col0.q +++ /dev/null @@ -1,2 +0,0 @@ --- TOK_ALLCOLREF -explain select * from (select * from (select * from src) a join (select * from src1) b on (a.key = b.key)) t; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col1.q deleted file mode 100644 index 9e8bcbd1bbf78..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col1.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.support.quoted.identifiers=none; --- TOK_TABLE_OR_COL -explain select * from (select `.*` from (select * from src) a join (select * from src1) b on (a.key = b.key)) t; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col2.q deleted file mode 100644 index 33d4aed3cd9ad..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col2.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.support.quoted.identifiers=none; --- DOT -explain select * from (select a.`[kv].*`, b.`[kv].*` from (select * from src) a join (select * from src1) b on (a.key = b.key)) t; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze.q deleted file mode 100644 index 874f5bfc14125..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze.q +++ /dev/null @@ -1 +0,0 @@ -analyze table srcpart compute statistics; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze1.q deleted file mode 100644 index 057a1a0b482ea..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze1.q +++ /dev/null @@ -1 +0,0 @@ -analyze table srcpart partition (key) compute statistics; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_non_existent_tbl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_non_existent_tbl.q deleted file mode 100644 index 78a97019f192e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_non_existent_tbl.q +++ /dev/null @@ -1 +0,0 @@ -analyze table nonexistent compute statistics; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_view.q deleted file mode 100644 index af4970f52e8be..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_view.q +++ /dev/null @@ -1,6 +0,0 @@ -DROP VIEW av; - -CREATE VIEW av AS SELECT * FROM src; - --- should fail: can't analyze a view...yet -ANALYZE TABLE av COMPUTE STATISTICS; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive1.q deleted file mode 100644 index a4b50f5e14107..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive1.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to archive a partition twice. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE srcpart_archived LIKE srcpart; - -INSERT OVERWRITE TABLE srcpart_archived PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE srcpart_archived ARCHIVE PARTITION (ds='2008-04-08', hr='12'); -ALTER TABLE srcpart_archived ARCHIVE PARTITION (ds='2008-04-08', hr='12'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive2.q deleted file mode 100644 index ff8dcb248568c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive2.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to unarchive a non-archived partition --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -drop table tstsrcpart; -create table tstsrcpart like srcpart; -insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='12') -select key, value from srcpart where ds='2008-04-08' and hr='12'; - -ALTER TABLE tstsrcpart UNARCHIVE PARTITION (ds='2008-04-08', hr='12'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive3.q deleted file mode 100644 index 53057daa1b622..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive3.q +++ /dev/null @@ -1,5 +0,0 @@ -set hive.archive.enabled = true; --- Tests archiving a table --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -ALTER TABLE srcpart ARCHIVE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive4.q deleted file mode 100644 index 56d6f1798debe..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive4.q +++ /dev/null @@ -1,5 +0,0 @@ -set hive.archive.enabled = true; --- Tests archiving multiple partitions --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -ALTER TABLE srcpart ARCHIVE PARTITION (ds='2008-04-08', hr='12') PARTITION (ds='2008-04-08', hr='11'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive5.q deleted file mode 100644 index 4f6dc8d72ceed..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive5.q +++ /dev/null @@ -1,5 +0,0 @@ -set hive.archive.enabled = true; --- Tests creating a partition where the partition value will collide with the --- a intermediate directory - -ALTER TABLE srcpart ADD PARTITION (ds='2008-04-08', hr='14_INTERMEDIATE_ORIGINAL') diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_corrupt.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_corrupt.q deleted file mode 100644 index 130b37b5c9d52..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_corrupt.q +++ /dev/null @@ -1,18 +0,0 @@ -USE default; - -set hive.archive.enabled = true; -set hive.enforce.bucketing = true; - -drop table tstsrcpart; - -create table tstsrcpart like srcpart; - --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20) --- The version of GzipCodec that is provided in Hadoop 0.20 silently ignores --- file format errors. However, versions of Hadoop that include --- HADOOP-6835 (e.g. 0.23 and 1.x) cause a Wrong File Format exception --- to be thrown during the LOAD step. This former behavior is tested --- in clientpositive/archive_corrupt.q - -load data local inpath '../../data/files/archive_corrupt.rc' overwrite into table tstsrcpart partition (ds='2008-04-08', hr='11'); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert1.q deleted file mode 100644 index deaff63d673a7..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert1.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to insert into archived partition. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE tstsrcpart LIKE srcpart; - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08', hr='12'); - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert2.q deleted file mode 100644 index d744f2487694c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert2.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to insert into archived partition. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE tstsrcpart LIKE srcpart; - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert3.q deleted file mode 100644 index c6cb142824c8d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert3.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to create partition inside of archived directory. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE tstsrcpart LIKE srcpart; - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='11') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='11'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert4.q deleted file mode 100644 index c36f3ef9e8775..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert4.q +++ /dev/null @@ -1,15 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to (possible) dynamic insert into archived partition. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE tstsrcpart LIKE srcpart; - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08', hr='12'); - -SET hive.exec.dynamic.partition=true; - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr) -SELECT key, value, hr FROM srcpart WHERE ds='2008-04-08' AND hr='12'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi1.q deleted file mode 100644 index 8c702ed008bfa..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi1.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to archive a partition twice. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE tstsrcpart LIKE srcpart; - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='11') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='11'; -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi2.q deleted file mode 100644 index d3cfb89c98749..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi2.q +++ /dev/null @@ -1,12 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to unarchive a non-archived partition group --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -drop table tstsrcpart; -create table tstsrcpart like srcpart; -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='11') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='11'; -insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='12') -select key, value from srcpart where ds='2008-04-08' and hr='12'; - -ALTER TABLE tstsrcpart UNARCHIVE PARTITION (ds='2008-04-08', hr='12'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi3.q deleted file mode 100644 index 75f5dfad47b37..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi3.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to archive outer partition group containing other partition inside. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE tstsrcpart LIKE srcpart; - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='11') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='11'; -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08', hr='12'); -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi4.q deleted file mode 100644 index abe0647ae6ee1..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi4.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to archive inner partition contained in archived partition group. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE tstsrcpart LIKE srcpart; - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='11') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='11'; -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08', hr='12'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi5.q deleted file mode 100644 index 71635e054a1e9..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi5.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to unarchive outer partition group containing other partition inside. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE tstsrcpart LIKE srcpart; - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='11') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='11'; -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08', hr='12'); -ALTER TABLE tstsrcpart UNARCHIVE PARTITION (ds='2008-04-08'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi6.q deleted file mode 100644 index 5bb1474fdc383..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi6.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to unarchive inner partition contained in archived partition group. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE tstsrcpart LIKE srcpart; - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='11') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='11'; -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); -ALTER TABLE tstsrcpart UNARCHIVE PARTITION (ds='2008-04-08', hr='12'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi7.q deleted file mode 100644 index db7f392737e97..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi7.q +++ /dev/null @@ -1,12 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to archive a partition group with custom locations. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE tstsrcpart LIKE srcpart; - -INSERT OVERWRITE TABLE tstsrcpart PARTITION (ds='2008-04-08', hr='11') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='11'; -ALTER TABLE tstsrcpart ADD PARTITION (ds='2008-04-08', hr='12') -LOCATION "${system:test.tmp.dir}/tstsrc"; - -ALTER TABLE tstsrcpart ARCHIVE PARTITION (ds='2008-04-08'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec1.q deleted file mode 100644 index d83b19d9fe31a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec1.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to archive a partition twice. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE srcpart_archived LIKE srcpart; - -INSERT OVERWRITE TABLE srcpart_archived PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE srcpart_archived ARCHIVE PARTITION (ds='2008-04-08', nonexistingpart='12'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec2.q deleted file mode 100644 index ed14bbf688d52..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec2.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to archive a partition twice. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE srcpart_archived LIKE srcpart; - -INSERT OVERWRITE TABLE srcpart_archived PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE srcpart_archived ARCHIVE PARTITION (hr='12'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec3.q deleted file mode 100644 index f27ad6d63b08e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec3.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to archive a partition twice. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE srcpart_archived LIKE srcpart; - -INSERT OVERWRITE TABLE srcpart_archived PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE srcpart_archived ARCHIVE PARTITION (); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec4.q deleted file mode 100644 index 491c2ac4596fc..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec4.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to archive a partition twice. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE srcpart_archived LIKE srcpart; - -INSERT OVERWRITE TABLE srcpart_archived PARTITION (ds='2008-04-08', hr='12') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE srcpart_archived ARCHIVE PARTITION (hr='12', ds='2008-04-08'); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec5.q deleted file mode 100644 index bb25ef2c7e0f1..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec5.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.archive.enabled = true; --- Tests trying to archive a partition twice. --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19) - -CREATE TABLE srcpart_archived (key string, value string) partitioned by (ds string, hr int, min int); - -INSERT OVERWRITE TABLE srcpart_archived PARTITION (ds='2008-04-08', hr='12', min='00') -SELECT key, value FROM srcpart WHERE ds='2008-04-08' AND hr='12'; - -ALTER TABLE srcpart_archived ARCHIVE PARTITION (ds='2008-04-08', min='00'); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addjar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addjar.q deleted file mode 100644 index a1709dae5f5b8..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addjar.q +++ /dev/null @@ -1,7 +0,0 @@ -set hive.security.authorization.enabled=true; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory; - --- running a sql query to initialize the authorization - not needed in real HS2 mode -show tables; - -add jar ${system:maven.local.repository}/org/apache/hive/hcatalog/hive-hcatalog-core/${system:hive.version}/hive-hcatalog-core-${system:hive.version}.jar; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addpartition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addpartition.q deleted file mode 100644 index 8abdd2b3cde8b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addpartition.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -set user.name=user1; --- check add partition without insert privilege -create table tpart(i int, j int) partitioned by (k string); - -set user.name=user2; -alter table tpart add partition (k = 'abc'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner.q deleted file mode 100644 index f716262e23bbb..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=user1; - --- check if alter table owner fails --- for now, alter db owner is allowed only for admin - -create database dbao; -alter database dbao set owner user user2; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner_default.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner_default.q deleted file mode 100644 index f9049350180ee..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner_default.q +++ /dev/null @@ -1,8 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=user1; - --- check if alter table owner fails -alter database default set owner user user1; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_all_role.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_all_role.q deleted file mode 100644 index de91e91923308..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_all_role.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set user.name=hive_admin_user; -set role ADMIN; -create role all; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_default_role.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_default_role.q deleted file mode 100644 index 42a42f65b28a9..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_default_role.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set user.name=hive_admin_user; -set role ADMIN; -create role default; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_none_role.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_none_role.q deleted file mode 100644 index 0d14cde6d5460..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_none_role.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set user.name=hive_admin_user; -set role ADMIN; -create role None; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_caseinsensitivity.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_caseinsensitivity.q deleted file mode 100644 index d5ea284f14749..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_caseinsensitivity.q +++ /dev/null @@ -1,17 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set user.name=hive_admin_user; -set role ADMIN; - -create role testrole; -show roles; -drop role TESTROLE; -show roles; -create role TESTROLE; -show roles; -grant role testROLE to user hive_admin_user; -set role testrolE; -set role adMin; -show roles; -create role TESTRoLE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func1.q deleted file mode 100644 index 02bbe090cfba7..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func1.q +++ /dev/null @@ -1,7 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=hive_test_user; - --- permanent function creation should fail for non-admin roles -create function perm_fn as 'org.apache.hadoop.hive.ql.udf.UDFAscii'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func2.q deleted file mode 100644 index 8760fa8d82259..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func2.q +++ /dev/null @@ -1,8 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=hive_test_user; - --- temp function creation should fail for non-admin roles -create temporary function temp_fn as 'org.apache.hadoop.hive.ql.udf.UDFAscii'; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_macro1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_macro1.q deleted file mode 100644 index c904a100c515c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_macro1.q +++ /dev/null @@ -1,8 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=hive_test_user; - --- temp macro creation should fail for non-admin roles -create temporary macro mymacro1(x double) x * x; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_role_no_admin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_role_no_admin.q deleted file mode 100644 index a84fe64bd618e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_role_no_admin.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; --- this test will fail because hive_test_user is not in admin role. -create role r1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_createview.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_createview.q deleted file mode 100644 index 9b1f2ea6c6acb..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_createview.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - --- check create view without select privileges -create table t1(i int); -set user.name=user1; -create view v1 as select * from t1; - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_ctas.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_ctas.q deleted file mode 100644 index 1cf74a365d79e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_ctas.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - --- check query without select privilege fails -create table t1(i int); - -set user.name=user1; -create table t2 as select * from t1; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_desc_table_nosel.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_desc_table_nosel.q deleted file mode 100644 index 47663c9bb93e3..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_desc_table_nosel.q +++ /dev/null @@ -1,14 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=user1; - --- check if alter table fails as different user -create table t1(i int); -desc t1; - -grant all on table t1 to user user2; -revoke select on table t1 from user user2; - -set user.name=user2; -desc t1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_dfs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_dfs.q deleted file mode 100644 index 7d47a7b64967b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_dfs.q +++ /dev/null @@ -1,7 +0,0 @@ -set hive.security.authorization.enabled=true; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory; - --- running a sql query to initialize the authorization - not needed in real HS2 mode -show tables; -dfs -ls ${system:test.tmp.dir}/ - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_disallow_transform.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_disallow_transform.q deleted file mode 100644 index 64b300c8d9b2f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_disallow_transform.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set role ALL; -SELECT TRANSFORM (*) USING 'cat' AS (key, value) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_cascade.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_cascade.q deleted file mode 100644 index edeae9b71d7ac..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_cascade.q +++ /dev/null @@ -1,22 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=user1; - --- ensure that drop database cascade works -create database dba1; -create table dba1.tab1(i int); -drop database dba1 cascade; - --- check if drop database fails if the db has a table for which user does not have permission -create database dba2; -create table dba2.tab2(i int); - -set user.name=hive_admin_user; -set role ADMIN; -alter database dba2 set owner user user2; - -set user.name=user2; -show current roles; -drop database dba2 cascade ; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_empty.q deleted file mode 100644 index 46d4d0f92c8e3..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_empty.q +++ /dev/null @@ -1,27 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=user1; - --- check if changing owner and dropping as other user works -create database dba1; - -set user.name=hive_admin_user; -set role ADMIN; -alter database dba1 set owner user user2; - -set user.name=user2; -show current roles; -drop database dba1; - - -set user.name=user1; --- check if dropping db as another user fails -show current roles; -create database dba2; - -set user.name=user2; -show current roles; - -drop database dba2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_role_no_admin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_role_no_admin.q deleted file mode 100644 index a7aa17f5abfcf..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_role_no_admin.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set user.name=hive_admin_user; -set role ADMIN; -show current roles; -create role r1; -set role ALL; -show current roles; -drop role r1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_droppartition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_droppartition.q deleted file mode 100644 index f05e9458fa804..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_droppartition.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/authz_drop_part_1; - --- check drop partition without delete privilege -create table tpart(i int, j int) partitioned by (k string); -alter table tpart add partition (k = 'abc') location 'file:${system:test.tmp.dir}/authz_drop_part_1' ; -set user.name=user1; -alter table tpart drop partition (k = 'abc'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_1.q deleted file mode 100644 index c38dab5eb702f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_1.q +++ /dev/null @@ -1,7 +0,0 @@ -create table authorization_fail_1 (key int, value string); -set hive.security.authorization.enabled=true; - -grant Create on table authorization_fail_1 to user hive_test_user; -grant Create on table authorization_fail_1 to user hive_test_user; - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_2.q deleted file mode 100644 index 341e44774d9c2..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_2.q +++ /dev/null @@ -1,7 +0,0 @@ -create table authorization_fail_2 (key int, value string) partitioned by (ds string); - -set hive.security.authorization.enabled=true; - -alter table authorization_fail_2 add partition (ds='2010'); - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_3.q deleted file mode 100644 index 6a56daa05fee8..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_3.q +++ /dev/null @@ -1,12 +0,0 @@ --- SORT_BEFORE_DIFF - -create table authorization_fail_3 (key int, value string) partitioned by (ds string); -set hive.security.authorization.enabled=true; - -grant Create on table authorization_fail_3 to user hive_test_user; -alter table authorization_fail_3 add partition (ds='2010'); - -show grant user hive_test_user on table authorization_fail_3; -show grant user hive_test_user on table authorization_fail_3 partition (ds='2010'); - -select key from authorization_fail_3 where ds='2010'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_4.q deleted file mode 100644 index f0cb6459a255b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_4.q +++ /dev/null @@ -1,15 +0,0 @@ --- SORT_BEFORE_DIFF - -create table authorization_fail_4 (key int, value string) partitioned by (ds string); - -set hive.security.authorization.enabled=true; -grant Alter on table authorization_fail_4 to user hive_test_user; -ALTER TABLE authorization_fail_4 SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="TRUE"); - -grant Create on table authorization_fail_4 to user hive_test_user; -alter table authorization_fail_4 add partition (ds='2010'); - -show grant user hive_test_user on table authorization_fail_4; -show grant user hive_test_user on table authorization_fail_4 partition (ds='2010'); - -select key from authorization_fail_4 where ds='2010'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_5.q deleted file mode 100644 index b4efab5667f6c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_5.q +++ /dev/null @@ -1,20 +0,0 @@ --- SORT_BEFORE_DIFF - -create table authorization_fail (key int, value string) partitioned by (ds string); -set hive.security.authorization.enabled=true; - -grant Alter on table authorization_fail to user hive_test_user; -ALTER TABLE authorization_fail SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="TRUE"); - -grant Create on table authorization_fail to user hive_test_user; -grant Select on table authorization_fail to user hive_test_user; -alter table authorization_fail add partition (ds='2010'); - -show grant user hive_test_user on table authorization_fail; -show grant user hive_test_user on table authorization_fail partition (ds='2010'); - -revoke Select on table authorization_fail partition (ds='2010') from user hive_test_user; - -show grant user hive_test_user on table authorization_fail partition (ds='2010'); - -select key from authorization_fail where ds='2010'; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_6.q deleted file mode 100644 index 977246948cad0..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_6.q +++ /dev/null @@ -1,6 +0,0 @@ --- SORT_BEFORE_DIFF - -create table authorization_part_fail (key int, value string) partitioned by (ds string); -set hive.security.authorization.enabled=true; - -ALTER TABLE authorization_part_fail SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="TRUE"); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_7.q deleted file mode 100644 index 492deed10bfeb..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_7.q +++ /dev/null @@ -1,17 +0,0 @@ --- SORT_BEFORE_DIFF - -create table authorization_fail (key int, value string); - -set hive.security.authorization.enabled=true; - -create role hive_test_role_fail; - -grant role hive_test_role_fail to user hive_test_user; -grant select on table authorization_fail to role hive_test_role_fail; -show role grant user hive_test_user; - -show grant role hive_test_role_fail on table authorization_fail; - -drop role hive_test_role_fail; - -select key from authorization_fail; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_create_db.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_create_db.q deleted file mode 100644 index d969e39027e99..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_create_db.q +++ /dev/null @@ -1,5 +0,0 @@ -set hive.security.authorization.enabled=true; - -create database db_to_fail; - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_drop_db.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_drop_db.q deleted file mode 100644 index 87719b0043e2e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_drop_db.q +++ /dev/null @@ -1,5 +0,0 @@ -set hive.security.authorization.enabled=false; -create database db_fail_to_drop; -set hive.security.authorization.enabled=true; - -drop database db_fail_to_drop; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q deleted file mode 100644 index f3c86b97ce76f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q +++ /dev/null @@ -1,14 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; - -set user.name=user1; --- current user has been set (comment line before the set cmd is resulting in parse error!!) - -CREATE TABLE table_priv_allf(i int); - --- grant insert to user2 WITH grant option -GRANT INSERT ON table_priv_allf TO USER user2 with grant option; - -set user.name=user2; --- try grant all to user3, without having all privileges -GRANT ALL ON table_priv_allf TO USER user3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_dup.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_dup.q deleted file mode 100644 index 7808cb3ec7b39..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_dup.q +++ /dev/null @@ -1,16 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; - -set user.name=user1; --- current user has been set (comment line before the set cmd is resulting in parse error!!) - -CREATE TABLE tauth_gdup(i int); - --- It should be possible to revert owners privileges -revoke SELECT ON tauth_gdup from user user1; - -show grant user user1 on table tauth_gdup; - --- Owner already has all privileges granted, another grant would become duplicate --- and result in error -GRANT INSERT ON tauth_gdup TO USER user1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q deleted file mode 100644 index 8dc8e45a79075..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; - -set user.name=user1; --- current user has been set (comment line before the set cmd is resulting in parse error!!) - -CREATE TABLE table_priv_gfail1(i int); - -set user.name=user2; --- try grant insert to user3 as user2 -GRANT INSERT ON table_priv_gfail1 TO USER user3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q deleted file mode 100644 index d51c1c3507eef..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q +++ /dev/null @@ -1,14 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; - -set user.name=user1; --- current user has been set (comment line before the set cmd is resulting in parse error!!) - -CREATE TABLE table_priv_gfail1(i int); - --- grant insert to user2 WITHOUT grant option -GRANT INSERT ON table_priv_gfail1 TO USER user2; - -set user.name=user2; --- try grant insert to user3 -GRANT INSERT ON table_priv_gfail1 TO USER user3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noinspriv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noinspriv.q deleted file mode 100644 index 2fa3cb260b07e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noinspriv.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - --- check insert without select priv -create table t1(i int); - -set user.name=user1; -create table user2tab(i int); -insert into table t1 select * from user2tab; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noselectpriv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noselectpriv.q deleted file mode 100644 index b9bee4ea40d40..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noselectpriv.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - --- check insert without select priv -create table t1(i int); - -set user.name=user1; -create table t2(i int); -insert into table t2 select * from t1; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v1.q deleted file mode 100644 index 2a1da23daeb18..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v1.q +++ /dev/null @@ -1,6 +0,0 @@ -create table if not exists authorization_invalid_v1 (key int, value string); -grant delete on table authorization_invalid_v1 to user hive_test_user; -drop table authorization_invalid_v1; - - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v2.q deleted file mode 100644 index 9c724085d9901..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v2.q +++ /dev/null @@ -1,5 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; - -create table if not exists authorization_invalid_v2 (key int, value string); -grant index on table authorization_invalid_v2 to user hive_test_user; -drop table authorization_invalid_v2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_rename.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_rename.q deleted file mode 100644 index 8a3300cb2e378..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_rename.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=user1; - --- check if alter table fails as different user -create table t1(i int); - -set user.name=user2; -alter table t1 rename to tnew1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_serdeprop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_serdeprop.q deleted file mode 100644 index 0172c4c74c82d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_serdeprop.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=user1; - --- check if alter table fails as different user -create table t1(i int); - -set user.name=user2; -ALTER TABLE t1 SET SERDEPROPERTIES ('field.delim' = ','); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_tab.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_tab.q deleted file mode 100644 index 2d0e52da008d8..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_tab.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=user1; - --- check if create table fails as different user -create table t1(i int); - -set user.name=user2; -drop table t1; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_view.q deleted file mode 100644 index 76bbab42b3750..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_view.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=user1; - --- check if create table fails as different user -create table t1(i int); -create view vt1 as select * from t1; - -set user.name=user2; -drop view vt1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_part.q deleted file mode 100644 index a654a2380c75c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_part.q +++ /dev/null @@ -1,37 +0,0 @@ --- SORT_BEFORE_DIFF - -create table authorization_part_fail (key int, value string) partitioned by (ds string); -ALTER TABLE authorization_part_fail SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="TRUE"); -create table src_auth as select * from src; -set hive.security.authorization.enabled=true; - -grant Create on table authorization_part_fail to user hive_test_user; -grant Update on table authorization_part_fail to user hive_test_user; -grant Drop on table authorization_part_fail to user hive_test_user; -grant select on table src_auth to user hive_test_user; - --- column grant to group - -grant select(key) on table authorization_part_fail to group hive_test_group1; -grant select on table authorization_part_fail to group hive_test_group1; - -show grant group hive_test_group1 on table authorization_part_fail; - -insert overwrite table authorization_part_fail partition (ds='2010') select key, value from src_auth; -show grant group hive_test_group1 on table authorization_part_fail(key) partition (ds='2010'); -show grant group hive_test_group1 on table authorization_part_fail partition (ds='2010'); -select key, value from authorization_part_fail where ds='2010' order by key limit 20; - -insert overwrite table authorization_part_fail partition (ds='2011') select key, value from src_auth; -show grant group hive_test_group1 on table authorization_part_fail(key) partition (ds='2011'); -show grant group hive_test_group1 on table authorization_part_fail partition (ds='2011'); -select key, value from authorization_part_fail where ds='2011' order by key limit 20; - -select key,value, ds from authorization_part_fail where ds>='2010' order by key, ds limit 20; - -revoke select on table authorization_part_fail partition (ds='2010') from group hive_test_group1; - -select key,value, ds from authorization_part_fail where ds>='2010' order by key, ds limit 20; - -drop table authorization_part_fail; -drop table src_auth; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_priv_current_role_neg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_priv_current_role_neg.q deleted file mode 100644 index bbf3b66970b6a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_priv_current_role_neg.q +++ /dev/null @@ -1,29 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set user.name=hive_admin_user; -set role ADMIN; - --- the test verifies that authorization is happening with privileges of the current roles - --- grant privileges with grant option for table to role2 -create role role2; -grant role role2 to user user2; -create table tpriv_current_role(i int); -grant all on table tpriv_current_role to role role2 with grant option; - -set user.name=user2; --- switch to user2 - --- by default all roles should be in current roles, and grant to new user should work -show current roles; -grant all on table tpriv_current_role to user user3; - -set role role2; --- switch to role2, grant should work -grant all on table tpriv_current_role to user user4; -show grant user user4 on table tpriv_current_role; - -set role PUBLIC; --- set role to public, should fail as role2 is not one of the current roles -grant all on table tpriv_current_role to user user5; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_create.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_create.q deleted file mode 100644 index 002389f203e25..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_create.q +++ /dev/null @@ -1 +0,0 @@ -create role PUBLIC; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_drop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_drop.q deleted file mode 100644 index 69c5a8de8b05f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_drop.q +++ /dev/null @@ -1 +0,0 @@ -drop role PUBLIC; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q deleted file mode 100644 index e19bf370fa077..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q +++ /dev/null @@ -1,14 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; - -set user.name=user1; --- current user has been set (comment line before the set cmd is resulting in parse error!!) - -CREATE TABLE table_priv_rfail1(i int); - --- grant insert to user2 -GRANT INSERT ON table_priv_rfail1 TO USER user2; - -set user.name=user3; --- try dropping the privilege as user3 -REVOKE INSERT ON TABLE table_priv_rfail1 FROM USER user2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q deleted file mode 100644 index 4b0cf3286ae71..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q +++ /dev/null @@ -1,18 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; - -set user.name=user1; --- current user has been set (comment line before the set cmd is resulting in parse error!!) - -CREATE TABLE table_priv_rfai2(i int); - --- grant insert to user2 -GRANT INSERT ON table_priv_rfai2 TO USER user2; -GRANT SELECT ON table_priv_rfai2 TO USER user3 WITH GRANT OPTION; - -set user.name=user3; --- grant select as user3 to user 2 -GRANT SELECT ON table_priv_rfai2 TO USER user2; - --- try dropping the privilege as user3 -REVOKE INSERT ON TABLE table_priv_rfai2 FROM USER user2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles1.q deleted file mode 100644 index a819d204f56b6..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles1.q +++ /dev/null @@ -1,12 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set user.name=hive_admin_user; -set role ADMIN; --- this is applicable to any security mode as check is in metastore -create role role1; -create role role2; -grant role role1 to role role2; - --- this will create a cycle -grant role role2 to role role1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles2.q deleted file mode 100644 index 423f030630b6c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles2.q +++ /dev/null @@ -1,24 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; - -set user.name=hive_admin_user; -set role ADMIN; --- this is applicable to any security mode as check is in metastore - -create role role1; - -create role role2; -grant role role2 to role role1; - -create role role3; -grant role role3 to role role2; - -create role role4; -grant role role4 to role role3; - -create role role5; -grant role role5 to role role4; - --- this will create a cycle in middle of the hierarchy -grant role role2 to role role4; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_grant.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_grant.q deleted file mode 100644 index c5c500a71251f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_grant.q +++ /dev/null @@ -1,22 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set user.name=hive_admin_user; - -set role ADMIN; - ----------------------------------------- --- role granting with admin option --- since user2 doesn't have admin option for role_noadmin, last grant should fail ----------------------------------------- - -create role role_noadmin; -create role src_role_wadmin; -grant src_role_wadmin to user user2 with admin option; -grant role_noadmin to user user2; -show role grant user user2; - - -set user.name=user2; -set role role_noadmin; -grant src_role_wadmin to user user3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_rolehierarchy_privs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_rolehierarchy_privs.q deleted file mode 100644 index d9f4c7cdb850b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_rolehierarchy_privs.q +++ /dev/null @@ -1,74 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -set user.name=hive_admin_user; -show current roles; -set role ADMIN; - ----------- --- create the following user, role mapping --- user1 -> role1 -> role2 -> role3 ----------- - -create role role1; -grant role1 to user user1; - -create role role2; -grant role2 to role role1; - -create role role3; -grant role3 to role role2; - - -create table t1(i int); -grant select on t1 to role role3; - -set user.name=user1; -show current roles; -select * from t1; - -set user.name=hive_admin_user; -show current roles; -grant select on t1 to role role2; - - -set user.name=user1; -show current roles; -select * from t1; - -set user.name=hive_admin_user; -set role ADMIN; -show current roles; -revoke select on table t1 from role role2; - - -create role role4; -grant role4 to user user1; -grant role3 to role role4;; - -set user.name=user1; -show current roles; -select * from t1; - -set user.name=hive_admin_user; -show current roles; -set role ADMIN; - --- Revoke role3 from hierarchy one at a time and check permissions --- after revoking from both, select should fail -revoke role3 from role role2; - -set user.name=user1; -show current roles; -select * from t1; - -set user.name=hive_admin_user; -show current roles; -set role ADMIN; -revoke role3 from role role4; - -set user.name=user1; -show current roles; -select * from t1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select.q deleted file mode 100644 index 39871793af398..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select.q +++ /dev/null @@ -1,9 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - --- check query without select privilege fails -create table t1(i int); - -set user.name=user1; -select * from t1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select_view.q deleted file mode 100644 index a4071cd0d4d87..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select_view.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - --- check create view without select privileges -create table t1(i int); -create view v1 as select * from t1; -set user.name=user1; -select * from v1; - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg1.q deleted file mode 100644 index 9ba3a82a5608e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg1.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; - --- an error should be thrown if 'set role ' is done for role that does not exist - -set role nosuchroleexists; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg2.q deleted file mode 100644 index 03f748fcc9b7a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg2.q +++ /dev/null @@ -1,16 +0,0 @@ -set hive.users.in.admin.role=hive_admin_user; -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set user.name=hive_admin_user; -set role ADMIN; - --- an error should be thrown if 'set role ' is done for role that does not exist - -create role rset_role_neg; -grant role rset_role_neg to user user2; - -set user.name=user2; -set role rset_role_neg; -set role public; -set role nosuchroleexists;; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_parts_nosel.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_parts_nosel.q deleted file mode 100644 index d8190de950de7..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_parts_nosel.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; -set user.name=user1; - --- check if alter table fails as different user -create table t_show_parts(i int) partitioned by (j string); - -set user.name=user2; -show partitions t_show_parts; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_no_admin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_no_admin.q deleted file mode 100644 index 2afe87fc30c9e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_no_admin.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; --- This test will fail because hive_test_user is not in admin role -show principals role1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_v1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_v1.q deleted file mode 100644 index 69cea2f2673f0..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_v1.q +++ /dev/null @@ -1,2 +0,0 @@ --- This test will fail because the command is not currently supported in auth mode v1 -show principals role1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_roles_no_admin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_roles_no_admin.q deleted file mode 100644 index 0fc9fca940c39..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_roles_no_admin.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; --- This test will fail because hive_test_user is not in admin role -show roles; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_truncate.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_truncate.q deleted file mode 100644 index 285600b23a149..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_truncate.q +++ /dev/null @@ -1,9 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - --- check add partition without insert privilege -create table t1(i int, j int); -set user.name=user1; -truncate table t1; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_add_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_add_partition.q deleted file mode 100644 index d82ac710cc3ba..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_add_partition.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/a_uri_add_part; -dfs -touchz ${system:test.tmp.dir}/a_uri_add_part/1.txt; -dfs -chmod 555 ${system:test.tmp.dir}/a_uri_add_part/1.txt; - -create table tpart(i int, j int) partitioned by (k string); -alter table tpart add partition (k = 'abc') location '${system:test.tmp.dir}/a_uri_add_part/'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_alterpart_loc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_alterpart_loc.q deleted file mode 100644 index d38ba74d9006a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_alterpart_loc.q +++ /dev/null @@ -1,16 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_alterpart_loc_perm; -dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_alterpart_loc; -dfs -touchz ${system:test.tmp.dir}/az_uri_alterpart_loc/1.txt; -dfs -chmod 555 ${system:test.tmp.dir}/az_uri_alterpart_loc/1.txt; - -create table tpart(i int, j int) partitioned by (k string); -alter table tpart add partition (k = 'abc') location '${system:test.tmp.dir}/az_uri_alterpart_loc_perm/'; - -alter table tpart partition (k = 'abc') set location '${system:test.tmp.dir}/az_uri_alterpart_loc/'; - - --- Attempt to set partition to location without permissions should fail diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_altertab_setloc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_altertab_setloc.q deleted file mode 100644 index c446b8636fb32..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_altertab_setloc.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_altertab_setloc; -dfs -touchz ${system:test.tmp.dir}/az_uri_altertab_setloc/1.txt; -dfs -chmod 555 ${system:test.tmp.dir}/az_uri_altertab_setloc/1.txt; - -create table t1(i int); - -alter table t1 set location '${system:test.tmp.dir}/az_uri_altertab_setloc/1.txt' - --- Attempt to set location of table to a location without permissions should fail diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q deleted file mode 100644 index c8e1fb43ee317..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/a_uri_crtab1; -dfs -touchz ${system:test.tmp.dir}/a_uri_crtab1/1.txt; -dfs -chmod 555 ${system:test.tmp.dir}/a_uri_crtab1/1.txt; - -create table t1(i int) location '${system:test.tmp.dir}/a_uri_crtab_ext'; - --- Attempt to create table with dir that does not have write permission should fail diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table_ext.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table_ext.q deleted file mode 100644 index c8549b4563b2f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table_ext.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/a_uri_crtab_ext; -dfs -touchz ${system:test.tmp.dir}/a_uri_crtab_ext/1.txt; -dfs -chmod 555 ${system:test.tmp.dir}/a_uri_crtab_ext/1.txt; - -create external table t1(i int) location '${system:test.tmp.dir}/a_uri_crtab_ext'; - --- Attempt to create table with dir that does not have write permission should fail diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_createdb.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_createdb.q deleted file mode 100644 index edfdf5a8fc407..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_createdb.q +++ /dev/null @@ -1,12 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_createdb; -dfs -touchz ${system:test.tmp.dir}/az_uri_createdb/1.txt; -dfs -chmod 300 ${system:test.tmp.dir}/az_uri_createdb/1.txt; - -create database az_test_db location '${system:test.tmp.dir}/az_uri_createdb/'; - --- Attempt to create db for dir without sufficient permissions should fail - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_export.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_export.q deleted file mode 100644 index 81763916a0b81..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_export.q +++ /dev/null @@ -1,22 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -set hive.test.mode=true; -set hive.test.mode.prefix=; -set hive.test.mode.nosamplelist=export_auth_uri; - - -create table export_auth_uri ( dep_id int comment "department id") - stored as textfile; - -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/export_auth_uri/temp; -dfs -rmr target/tmp/ql/test/data/exports/export_auth_uri; - - -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/export_auth_uri/; -dfs -chmod 555 target/tmp/ql/test/data/exports/export_auth_uri; - -export table export_auth_uri to 'ql/test/data/exports/export_auth_uri'; - --- Attempt to export to location without sufficient permissions should fail diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_import.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_import.q deleted file mode 100644 index 4ea4dc0a4747a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_import.q +++ /dev/null @@ -1,25 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -set hive.test.mode=true; -set hive.test.mode.prefix=; -set hive.test.mode.nosamplelist=import_auth_uri; - - -create table import_auth_uri ( dep_id int comment "department id") - stored as textfile; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/import_auth_uri/temp; -dfs -rmr target/tmp/ql/test/data/exports/import_auth_uri; -export table import_auth_uri to 'ql/test/data/exports/import_auth_uri'; -drop table import_auth_uri; - -dfs -touchz target/tmp/ql/test/data/exports/import_auth_uri/1.txt; -dfs -chmod 555 target/tmp/ql/test/data/exports/import_auth_uri/1.txt; - -create database importer; -use importer; - -import from 'ql/test/data/exports/import_auth_uri'; - --- Attempt to import from location without sufficient permissions should fail diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_index.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_index.q deleted file mode 100644 index 1a8f9cb2ad197..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_index.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_index; -dfs -touchz ${system:test.tmp.dir}/az_uri_index/1.txt; -dfs -chmod 555 ${system:test.tmp.dir}/az_uri_index/1.txt; - - -create table t1(i int); -create index idt1 on table t1 (i) as 'COMPACT' WITH DEFERRED REBUILD LOCATION '${system:test.tmp.dir}/az_uri_index/'; - --- Attempt to use location for index that does not have permissions should fail diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert.q deleted file mode 100644 index 81b6e522c1abb..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert.q +++ /dev/null @@ -1,14 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_insert; -dfs -touchz ${system:test.tmp.dir}/az_uri_insert/1.txt; -dfs -chmod 555 ${system:test.tmp.dir}/az_uri_insert/1.txt; - -create table t1(i int, j int); - -insert overwrite directory '${system:test.tmp.dir}/az_uri_insert/' select * from t1; - --- Attempt to insert into uri without permissions should fail - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert_local.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert_local.q deleted file mode 100644 index 0a2fd8919f455..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert_local.q +++ /dev/null @@ -1,14 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_insert_local; -dfs -touchz ${system:test.tmp.dir}/az_uri_insert_local/1.txt; -dfs -chmod 555 ${system:test.tmp.dir}/az_uri_insert_local/1.txt; - -create table t1(i int, j int); - -insert overwrite local directory '${system:test.tmp.dir}/az_uri_insert_local/' select * from t1; - --- Attempt to insert into uri without permissions should fail - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_load_data.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_load_data.q deleted file mode 100644 index 6af41f0cdaa23..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_load_data.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; -set hive.security.authorization.enabled=true; - -dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/authz_uri_load_data; -dfs -touchz ${system:test.tmp.dir}/authz_uri_load_data/1.txt; -dfs -chmod 555 ${system:test.tmp.dir}/authz_uri_load_data/1.txt; - -create table t1(i int); -load data inpath 'pfile:${system:test.tmp.dir}/authz_uri_load_data/' overwrite into table t1; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_create_tbl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_create_tbl.q deleted file mode 100644 index d8beac370d4b6..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_create_tbl.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; -set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; - -set hive.security.authorization.enabled=true; -set user.name=user33; -create database db23221; -use db23221; - -set user.name=user44; -create table twew221(a string); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_grant_public.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_grant_public.q deleted file mode 100644 index bfd3165237774..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_grant_public.q +++ /dev/null @@ -1 +0,0 @@ -grant role PUBLIC to user hive_test_user; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_revoke_public.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_revoke_public.q deleted file mode 100644 index 2b29822371b19..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_revoke_public.q +++ /dev/null @@ -1 +0,0 @@ -revoke role PUBLIC from user hive_test_user; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/autolocal1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/autolocal1.q deleted file mode 100644 index bd1c9d6e15a76..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/autolocal1.q +++ /dev/null @@ -1,15 +0,0 @@ -set mapred.job.tracker=abracadabra; -set hive.exec.mode.local.auto.inputbytes.max=1; -set hive.exec.mode.local.auto=true; - --- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20) --- hadoop0.23 changes the behavior of JobClient initialization --- in hadoop0.20, JobClient initialization tries to get JobTracker's address --- this throws the expected IllegalArgumentException --- in hadoop0.23, JobClient initialization only initializes cluster --- and get user group information --- not attempts to get JobTracker's address --- no IllegalArgumentException thrown in JobClient Initialization --- an exception is thrown when JobClient submitJob - -SELECT key FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_exec_hooks.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_exec_hooks.q deleted file mode 100644 index 709d8d9c85442..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_exec_hooks.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.exec.pre.hooks="org.this.is.a.bad.class"; - -EXPLAIN -SELECT x.* FROM SRC x LIMIT 20; - -SELECT x.* FROM SRC x LIMIT 20; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_indextype.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_indextype.q deleted file mode 100644 index 8f5bf42664b96..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_indextype.q +++ /dev/null @@ -1 +0,0 @@ -CREATE INDEX srcpart_index_proj ON TABLE srcpart(key) AS 'UNKNOWN' WITH DEFERRED REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_sample_clause.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_sample_clause.q deleted file mode 100644 index fd6769827b829..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_sample_clause.q +++ /dev/null @@ -1,6 +0,0 @@ -CREATE TABLE dest1(key INT, value STRING, dt STRING, hr STRING) STORED AS TEXTFILE; - -INSERT OVERWRITE TABLE dest1 SELECT s.* -FROM srcpart TABLESAMPLE (BUCKET 1 OUT OF 2) s -WHERE s.ds='2008-04-08' and s.hr='11'; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q deleted file mode 100644 index 6bebb8942d613..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q +++ /dev/null @@ -1,42 +0,0 @@ -CREATE TABLE srcbucket_mapjoin_part (key int, value string) - partitioned by (ds string) CLUSTERED BY (key) INTO 3 BUCKETS - STORED AS TEXTFILE; -load data local inpath '../../data/files/srcbucket20.txt' - INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../../data/files/srcbucket21.txt' - INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../../data/files/srcbucket22.txt' - INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); - -CREATE TABLE srcbucket_mapjoin_part_2 (key int, value string) - partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS - STORED AS TEXTFILE; -load data local inpath '../../data/files/srcbucket22.txt' - INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); -load data local inpath '../../data/files/srcbucket23.txt' - INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); - --- The number of buckets in the 2 tables above (being joined later) dont match. --- Throw an error if the user requested a bucketed mapjoin to be enforced. --- In the default case (hive.enforce.bucketmapjoin=false), the query succeeds --- even though mapjoin is not being performed - -explain -select a.key, a.value, b.value -from srcbucket_mapjoin_part a join srcbucket_mapjoin_part_2 b -on a.key=b.key and a.ds="2008-04-08" and b.ds="2008-04-08"; - -set hive.optimize.bucketmapjoin = true; - -explain -select /*+mapjoin(b)*/ a.key, a.value, b.value -from srcbucket_mapjoin_part a join srcbucket_mapjoin_part_2 b -on a.key=b.key and a.ds="2008-04-08" and b.ds="2008-04-08"; - -set hive.enforce.bucketmapjoin=true; - -explain -select /*+mapjoin(b)*/ a.key, a.value, b.value -from srcbucket_mapjoin_part a join srcbucket_mapjoin_part_2 b -on a.key=b.key and a.ds="2008-04-08" and b.ds="2008-04-08"; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q deleted file mode 100644 index 802fcd903c0ac..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q +++ /dev/null @@ -1,20 +0,0 @@ --- Although the user has specified a bucketed map-join, the number of buckets in the table --- do not match the number of files -drop table table1; -drop table table2; - -create table table1(key string, value string) clustered by (key, value) -into 2 BUCKETS stored as textfile; -create table table2(key string, value string) clustered by (value, key) -into 2 BUCKETS stored as textfile; - -load data local inpath '../../data/files/T1.txt' overwrite into table table1; - -load data local inpath '../../data/files/T1.txt' overwrite into table table2; -load data local inpath '../../data/files/T2.txt' overwrite into table table2; - -set hive.optimize.bucketmapjoin = true; -set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; - -select /*+ mapjoin(b) */ count(*) from table1 a join table2 b on a.key=b.key and a.value=b.value; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q deleted file mode 100644 index ac5abebb0b4b6..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q +++ /dev/null @@ -1,24 +0,0 @@ --- Although the user has specified a bucketed map-join, the number of buckets in the table --- do not match the number of files -drop table table1; -drop table table2; - -create table table1(key string, value string) partitioned by (ds string) clustered by (key, value) -into 2 BUCKETS stored as textfile; -create table table2(key string, value string) clustered by (value, key) -into 2 BUCKETS stored as textfile; - -load data local inpath '../../data/files/T1.txt' overwrite into table table1 partition (ds='1'); -load data local inpath '../../data/files/T2.txt' overwrite into table table1 partition (ds='1'); - -load data local inpath '../../data/files/T1.txt' overwrite into table table1 partition (ds='2'); - -load data local inpath '../../data/files/T1.txt' overwrite into table table2; -load data local inpath '../../data/files/T2.txt' overwrite into table table2; - -set hive.optimize.bucketmapjoin = true; -set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; - -select /*+ mapjoin(b) */ count(*) from table1 a join table2 b -on a.key=b.key and a.value=b.value and a.ds is not null; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cachingprintstream.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cachingprintstream.q deleted file mode 100644 index d57a4517f00fb..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cachingprintstream.q +++ /dev/null @@ -1,8 +0,0 @@ -set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifyCachingPrintStreamHook; -set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyCachingPrintStreamHook; - -SELECT count(*) FROM src; -FROM src SELECT TRANSFORM (key, value) USING 'FAKE_SCRIPT_SHOULD_NOT_EXIST' AS key, value; - -set hive.exec.failure.hooks=; -set hive.exec.post.hooks=; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cluster_tasklog_retrieval.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cluster_tasklog_retrieval.q deleted file mode 100644 index bc980448a9e22..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cluster_tasklog_retrieval.q +++ /dev/null @@ -1,6 +0,0 @@ --- TaskLog retrieval upon Null Pointer Exception in Cluster - -CREATE TEMPORARY FUNCTION evaluate_npe AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFEvaluateNPE'; - -FROM src -SELECT evaluate_npe(src.key) LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbydistributeby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbydistributeby.q deleted file mode 100644 index 4c6a9b38d785d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbydistributeby.q +++ /dev/null @@ -1,8 +0,0 @@ -CREATE TABLE dest1(key INT, ten INT, one INT, value STRING) STORED AS TEXTFILE; - -FROM src -INSERT OVERWRITE TABLE dest1 -MAP src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value -USING 'cat' AS (tkey, ten, one, tvalue) -CLUSTER BY tvalue, tkey -DISTRIBUTE BY tvalue, tkey; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbyorderby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbyorderby.q deleted file mode 100644 index d9ee9b9d262d0..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbyorderby.q +++ /dev/null @@ -1,5 +0,0 @@ -FROM src -MAP src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value -USING 'cat' AS (tkey, ten, one, tvalue) -CLUSTER BY tvalue, tkey -ORDER BY ten, one; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbysortby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbysortby.q deleted file mode 100644 index 7b4e744ba66df..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbysortby.q +++ /dev/null @@ -1,8 +0,0 @@ -CREATE TABLE dest1(key INT, ten INT, one INT, value STRING) STORED AS TEXTFILE; - -FROM src -INSERT OVERWRITE TABLE dest1 -MAP src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value -USING 'cat' AS (tkey, ten, one, tvalue) -CLUSTER BY tvalue, tkey -SORT BY ten, one; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern2.q deleted file mode 100644 index 9ed8944d2bb6c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern2.q +++ /dev/null @@ -1,3 +0,0 @@ -EXPLAIN -SELECT x.key, x.value as v1, y.* FROM SRC x JOIN SRC y ON (x.key = y.key) CLUSTER BY key; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern3.q deleted file mode 100644 index 23f73667edf5f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern3.q +++ /dev/null @@ -1,2 +0,0 @@ -EXPLAIN -SELECT x.key as k1, x.value FROM SRC x CLUSTER BY x.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern4.q deleted file mode 100644 index 3a9b45ca60576..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern4.q +++ /dev/null @@ -1,2 +0,0 @@ -EXPLAIN -SELECT x.key as k1, x.value FROM SRC x CLUSTER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_change_skewedcol_type1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_change_skewedcol_type1.q deleted file mode 100644 index 9a3e0b2efe693..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_change_skewedcol_type1.q +++ /dev/null @@ -1,5 +0,0 @@ -set hive.mapred.supports.subdirectories=true; - -CREATE TABLE skewedtable (key STRING, value STRING) SKEWED BY (key) ON (1,5,6); - -ALTER TABLE skewedtable CHANGE key key INT; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename1.q deleted file mode 100644 index d99b821802df7..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename1.q +++ /dev/null @@ -1,6 +0,0 @@ -drop table tstsrc; -create table tstsrc like src; -insert overwrite table tstsrc -select key, value from src; - -alter table tstsrc change src_not_exist key_value string; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename2.q deleted file mode 100644 index cccc8ad54e300..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename2.q +++ /dev/null @@ -1,6 +0,0 @@ -drop table tstsrc; -create table tstsrc like src; -insert overwrite table tstsrc -select key, value from src; - -alter table tstsrc change key value string; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename3.q deleted file mode 100644 index 91c9537a99ad5..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename3.q +++ /dev/null @@ -1 +0,0 @@ -alter table src change key key; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename4.q deleted file mode 100644 index dd89a5a10b227..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename4.q +++ /dev/null @@ -1,6 +0,0 @@ -drop table tstsrc; -create table tstsrc like src; -insert overwrite table tstsrc -select key, value from src; - -alter table tstsrc change key key2 string after key_value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename5.q deleted file mode 100644 index 3827b83361fbb..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename5.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.mapred.supports.subdirectories=true; - -CREATE TABLE skewedtable (key STRING, value STRING) SKEWED BY (key) ON (1,5,6); - -ALTER TABLE skewedtable CHANGE key key_new STRING; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q deleted file mode 100644 index b4887c4115854..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q +++ /dev/null @@ -1,16 +0,0 @@ -DROP TABLE Employee_Part; - -CREATE TABLE Employee_Part(employeeID int, employeeName String) partitioned by (employeeSalary double, country string) -row format delimited fields terminated by '|' stored as textfile; - -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='USA'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='UK'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='USA'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='4000.0', country='USA'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3500.0', country='UK'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='UK'); - --- dynamic partitioning syntax -explain -analyze table Employee_Part partition (employeeSalary='4000.0', country) compute statistics for columns employeeName, employeeID; -analyze table Employee_Part partition (employeeSalary='4000.0', country) compute statistics for columns employeeName, employeeID; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q deleted file mode 100644 index 2f8e9271ddd34..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q +++ /dev/null @@ -1,16 +0,0 @@ -DROP TABLE Employee_Part; - -CREATE TABLE Employee_Part(employeeID int, employeeName String) partitioned by (employeeSalary double, country string) -row format delimited fields terminated by '|' stored as textfile; - -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='USA'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='UK'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='USA'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='4000.0', country='USA'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3500.0', country='UK'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='UK'); - --- don't specify all partitioning keys -explain -analyze table Employee_Part partition (employeeSalary='2000.0') compute statistics for columns employeeID; -analyze table Employee_Part partition (employeeSalary='2000.0') compute statistics for columns employeeID; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q deleted file mode 100644 index 34f91fc8d1de8..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q +++ /dev/null @@ -1,16 +0,0 @@ -DROP TABLE Employee_Part; - -CREATE TABLE Employee_Part(employeeID int, employeeName String) partitioned by (employeeSalary double, country string) -row format delimited fields terminated by '|' stored as textfile; - -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='USA'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='UK'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='USA'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='4000.0', country='USA'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3500.0', country='UK'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='UK'); - --- specify invalid values for the partitioning keys -explain -analyze table Employee_Part partition (employeeSalary='4000.0', country='Canada') compute statistics for columns employeeName, employeeID; -analyze table Employee_Part partition (employeeSalary='4000.0', country='Canada') compute statistics for columns employeeName, employeeID; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q deleted file mode 100644 index 49d89dd12132b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q +++ /dev/null @@ -1,16 +0,0 @@ -DROP TABLE Employee_Part; - -CREATE TABLE Employee_Part(employeeID int, employeeName String) partitioned by (employeeSalary double, country string) -row format delimited fields terminated by '|' stored as textfile; - -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='USA'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='UK'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='USA'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='4000.0', country='USA'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3500.0', country='UK'); -LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='UK'); - --- specify partitioning clause multiple times -explain -analyze table Employee_Part partition (employeeSalary='4000.0', country='USA') partition(employeeSalary='2000.0', country='USA') compute statistics for columns employeeName, employeeID; -analyze table Employee_Part partition (employeeSalary='4000.0', country='USA') partition(employeeSalary='2000.0', country='USA') compute statistics for columns employeeName, employeeID; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl.q deleted file mode 100644 index a4e0056bff370..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl.q +++ /dev/null @@ -1,22 +0,0 @@ - -DROP TABLE IF EXISTS UserVisits_web_text_none; - -CREATE TABLE UserVisits_web_text_none ( - sourceIP string, - destURL string, - visitDate string, - adRevenue float, - userAgent string, - cCode string, - lCode string, - sKeyword string, - avgTimeOnSite int) -row format delimited fields terminated by '|' stored as textfile; - -LOAD DATA LOCAL INPATH "../../data/files/UserVisits.dat" INTO TABLE UserVisits_web_text_none; - -explain -analyze table UserVisits_web_text_none compute statistics for columns destIP; - -analyze table UserVisits_web_text_none compute statistics for columns destIP; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q deleted file mode 100644 index 85a5f0a021940..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q +++ /dev/null @@ -1,17 +0,0 @@ - -DROP TABLE IF EXISTS table_complex_type; - -CREATE TABLE table_complex_type ( - a STRING, - b ARRAY, - c ARRAY>, - d MAP> - ) STORED AS TEXTFILE; - -LOAD DATA LOCAL INPATH '../../data/files/create_nested_type.txt' OVERWRITE INTO TABLE table_complex_type; - - -explain -analyze table table_complex_type compute statistics for columns d; - -analyze table table_complex_type compute statistics for columns d; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q deleted file mode 100644 index a4e0056bff370..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q +++ /dev/null @@ -1,22 +0,0 @@ - -DROP TABLE IF EXISTS UserVisits_web_text_none; - -CREATE TABLE UserVisits_web_text_none ( - sourceIP string, - destURL string, - visitDate string, - adRevenue float, - userAgent string, - cCode string, - lCode string, - sKeyword string, - avgTimeOnSite int) -row format delimited fields terminated by '|' stored as textfile; - -LOAD DATA LOCAL INPATH "../../data/files/UserVisits.dat" INTO TABLE UserVisits_web_text_none; - -explain -analyze table UserVisits_web_text_none compute statistics for columns destIP; - -analyze table UserVisits_web_text_none compute statistics for columns destIP; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compare_double_bigint.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compare_double_bigint.q deleted file mode 100644 index 8ee4b277cbf72..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compare_double_bigint.q +++ /dev/null @@ -1,5 +0,0 @@ -set hive.mapred.mode=strict; - --- This should fail until we fix the issue with precision when casting a bigint to a double - -select * from src where cast(1 as bigint) = 1.0 limit 10; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compare_string_bigint.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compare_string_bigint.q deleted file mode 100644 index 810f65d4d2b44..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compare_string_bigint.q +++ /dev/null @@ -1,5 +0,0 @@ -set hive.mapred.mode=strict; - ---This should fail until we fix the issue with precision when casting a bigint to a double - -select * from src where cast(1 as bigint) = '1' limit 10; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compile_processor.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compile_processor.q deleted file mode 100644 index c314a940f95c2..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compile_processor.q +++ /dev/null @@ -1,8 +0,0 @@ - -compile `import org.apache.hadoop.hive.ql.exec.UDF \; -public class Pyth extsfgsfgfsends UDF { - public double evaluate(double a, double b){ - return Math.sqrt((a*a) + (b*b)) \; - } -} ` AS GROOVY NAMED Pyth.groovy; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compute_stats_long.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compute_stats_long.q deleted file mode 100644 index 5974811280350..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compute_stats_long.q +++ /dev/null @@ -1,7 +0,0 @@ -create table tab_int(a int); - --- insert some data -LOAD DATA LOCAL INPATH "../../data/files/int.txt" INTO TABLE tab_int; - --- compute stats should raise an error since the number of bit vectors > 1024 -select compute_stats(a, 10000) from tab_int; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_class.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_class.q deleted file mode 100644 index 3b71e00b2eaa2..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_class.q +++ /dev/null @@ -1 +0,0 @@ -create function default.badfunc as 'my.nonexistent.class'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_db.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_db.q deleted file mode 100644 index ae95391edd3e5..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_db.q +++ /dev/null @@ -1 +0,0 @@ -create function nonexistentdb.badfunc as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonudf_class.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonudf_class.q deleted file mode 100644 index 2083064593299..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonudf_class.q +++ /dev/null @@ -1 +0,0 @@ -create function default.badfunc as 'java.lang.String'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_insert_outputformat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_insert_outputformat.q deleted file mode 100644 index a052663055ef6..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_insert_outputformat.q +++ /dev/null @@ -1,11 +0,0 @@ - - -CREATE TABLE table_test_output_format(key INT, value STRING) STORED AS - INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat' - OUTPUTFORMAT 'org.apache.hadoop.mapred.MapFileOutputFormat'; - -FROM src -INSERT OVERWRITE TABLE table_test_output_format SELECT src.key, src.value LIMIT 10; - -describe table_test_output_format; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view1.q deleted file mode 100644 index c332278b84f68..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view1.q +++ /dev/null @@ -1,6 +0,0 @@ --- Cannot add or drop partition columns with CREATE OR REPLACE VIEW if partitions currently exist (must specify partition columns) - -drop view v; -create view v partitioned on (ds, hr) as select * from srcpart; -alter view v add partition (ds='1',hr='2'); -create or replace view v as select * from srcpart; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view2.q deleted file mode 100644 index b53dd07ce8ae6..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view2.q +++ /dev/null @@ -1,6 +0,0 @@ --- Cannot add or drop partition columns with CREATE OR REPLACE VIEW if partitions currently exist - -drop view v; -create view v partitioned on (ds, hr) as select * from srcpart; -alter view v add partition (ds='1',hr='2'); -create or replace view v partitioned on (hr) as select * from srcpart; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view3.q deleted file mode 100644 index d6fa7785dfa95..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view3.q +++ /dev/null @@ -1,3 +0,0 @@ --- Existing table is not a view - -create or replace view src as select ds, hr from srcpart; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view4.q deleted file mode 100644 index 12b6059b9e3ee..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view4.q +++ /dev/null @@ -1,5 +0,0 @@ --- View must have at least one non-partition column. - -drop view v; -create view v partitioned on (ds, hr) as select * from srcpart; -create or replace view v partitioned on (ds, hr) as select ds, hr from srcpart; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view5.q deleted file mode 100644 index 4eb9c94896d8f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view5.q +++ /dev/null @@ -1,5 +0,0 @@ --- Can't combine IF NOT EXISTS and OR REPLACE. - -drop view v; -create view v partitioned on (ds, hr) as select * from srcpart; -create or replace view if not exists v as select * from srcpart; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view6.q deleted file mode 100644 index a2f916fb26528..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view6.q +++ /dev/null @@ -1,5 +0,0 @@ --- Can't update view to have an invalid definition - -drop view v; -create view v partitioned on (ds, hr) as select * from srcpart; -create or replace view v partitioned on (ds, hr) as blah; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view7.q deleted file mode 100644 index 765a96572a04f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view7.q +++ /dev/null @@ -1,7 +0,0 @@ --- Can't update view to have a view cycle (1) - -drop view v; -create view v1 partitioned on (ds, hr) as select * from srcpart; -create view v2 partitioned on (ds, hr) as select * from v1; -create view v3 partitioned on (ds, hr) as select * from v2; -create or replace view v1 partitioned on (ds, hr) as select * from v3; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view8.q deleted file mode 100644 index f3a59b1d07be7..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view8.q +++ /dev/null @@ -1,5 +0,0 @@ --- Can't update view to have a view cycle (2) - -drop view v; -create view v1 partitioned on (ds, hr) as select * from srcpart; -create or replace view v1 partitioned on (ds, hr) as select * from v1; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_col_name_value_no_mismatch.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_col_name_value_no_mismatch.q deleted file mode 100644 index 1d6574e739602..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_col_name_value_no_mismatch.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.mapred.supports.subdirectories=true; - -CREATE TABLE skewed_table (key STRING, value STRING) SKEWED BY (key) ON ((1),(5,8),(6)); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_dup_col_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_dup_col_name.q deleted file mode 100644 index 726f6dd1dfcf6..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_dup_col_name.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.mapred.supports.subdirectories=true; - -CREATE TABLE skewed_table (key STRING, value STRING) SKEWED BY (key,key) ON ((1),(5),(6)); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_failure_invalid_col_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_failure_invalid_col_name.q deleted file mode 100644 index 30dd4181653d5..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_failure_invalid_col_name.q +++ /dev/null @@ -1,4 +0,0 @@ -set hive.mapred.supports.subdirectories=true; - -CREATE TABLE skewed_table (key STRING, value STRING) SKEWED BY (key_non) ON ((1),(5),(6)); - \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure1.q deleted file mode 100644 index e87c12b8a1fe6..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure1.q +++ /dev/null @@ -1 +0,0 @@ -create table table_in_database_creation_not_exist.test as select * from src limit 1; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure2.q deleted file mode 100644 index 0bddae066450e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure2.q +++ /dev/null @@ -1 +0,0 @@ -create table `table_in_database_creation_not_exist.test` as select * from src limit 1; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure3.q deleted file mode 100644 index 9f9f5f64dfd94..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure3.q +++ /dev/null @@ -1 +0,0 @@ -create table table_in_database_creation_not_exist.test (a string); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure4.q deleted file mode 100644 index 67745e011141a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure4.q +++ /dev/null @@ -1 +0,0 @@ -create table `table_in_database_creation_not_exist.test` (a string); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_wrong_regex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_wrong_regex.q deleted file mode 100644 index dc91c9c9ef05e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_wrong_regex.q +++ /dev/null @@ -1,4 +0,0 @@ -drop table aa; -create table aa ( test STRING ) - ROW FORMAT SERDE 'org.apache.hadoop.hive.contrib.serde2.RegexSerDe' - WITH SERDEPROPERTIES ("input.regex" = "[^\\](.*)", "output.format.string" = "$1s"); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_udaf_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_udaf_failure.q deleted file mode 100644 index e0bb408a64f2f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_udaf_failure.q +++ /dev/null @@ -1,6 +0,0 @@ -CREATE TEMPORARY FUNCTION test_udaf AS 'org.apache.hadoop.hive.ql.udf.UDAFWrongArgLengthForTestCase'; - -EXPLAIN -SELECT test_udaf(length(src.value)) FROM src; - -SELECT test_udaf(length(src.value)) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_unknown_genericudf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_unknown_genericudf.q deleted file mode 100644 index 07010c11c7d40..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_unknown_genericudf.q +++ /dev/null @@ -1 +0,0 @@ -CREATE TEMPORARY FUNCTION dummy_genericudf AS 'org.apache.hadoop.hive.ql.udf.generic.DummyGenericUDF'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_unknown_udf_udaf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_unknown_udf_udaf.q deleted file mode 100644 index a243fff033c4d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_unknown_udf_udaf.q +++ /dev/null @@ -1 +0,0 @@ -CREATE TEMPORARY FUNCTION dummy_function AS 'org.apache.hadoop.hive.ql.udf.DummyFunction'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure1.q deleted file mode 100644 index c9060c6766491..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure1.q +++ /dev/null @@ -1,6 +0,0 @@ - -DROP VIEW xxx12; - --- views and tables share the same namespace -CREATE TABLE xxx12(key int); -CREATE VIEW xxx12 AS SELECT key FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure2.q deleted file mode 100644 index 6fdcd4a9d377d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure2.q +++ /dev/null @@ -1,6 +0,0 @@ - -DROP VIEW xxx4; - --- views and tables share the same namespace -CREATE VIEW xxx4 AS SELECT key FROM src; -CREATE TABLE xxx4(key int); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure3.q deleted file mode 100644 index ad5fc499edf94..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure3.q +++ /dev/null @@ -1,5 +0,0 @@ -DROP VIEW xxx13; - --- number of explicit view column defs must match underlying SELECT -CREATE VIEW xxx13(x,y,z) AS -SELECT key FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure4.q deleted file mode 100644 index eecde65e1137e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure4.q +++ /dev/null @@ -1,5 +0,0 @@ -DROP VIEW xxx5; - --- duplicate column names are illegal -CREATE VIEW xxx5(x,x) AS -SELECT key,value FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure5.q deleted file mode 100644 index f720899168735..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure5.q +++ /dev/null @@ -1,9 +0,0 @@ -DROP VIEW xxx14; - --- Ideally (and according to SQL:200n), this should actually be legal, --- but since internally we impose the new column descriptors by --- reference to underlying name rather than position, we have to make --- it illegal. There's an easy workaround (provide the unique names --- via direct column aliases, e.g. SELECT key AS x, key AS y) -CREATE VIEW xxx14(x,y) AS -SELECT key,key FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure6.q deleted file mode 100644 index 57f52a8af149d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure6.q +++ /dev/null @@ -1,6 +0,0 @@ -DROP VIEW xxx15; - --- should fail: baz is not a column -CREATE VIEW xxx15 -PARTITIONED ON (baz) -AS SELECT key FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure7.q deleted file mode 100644 index 00d7f9fbf4eda..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure7.q +++ /dev/null @@ -1,6 +0,0 @@ -DROP VIEW xxx16; - --- should fail: must have at least one non-partitioning column -CREATE VIEW xxx16 -PARTITIONED ON (key) -AS SELECT key FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure8.q deleted file mode 100644 index 08291826d978b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure8.q +++ /dev/null @@ -1,6 +0,0 @@ -DROP VIEW xxx17; - --- should fail: partitioning key must be at end -CREATE VIEW xxx17 -PARTITIONED ON (key) -AS SELECT key,value FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure9.q deleted file mode 100644 index d7d44a49c393a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure9.q +++ /dev/null @@ -1,6 +0,0 @@ -DROP VIEW xxx18; - --- should fail: partitioning columns out of order -CREATE VIEW xxx18 -PARTITIONED ON (value,key) -AS SELECT key+1 as k2,key,value FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ctas.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ctas.q deleted file mode 100644 index 507a7a76b1ee7..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ctas.q +++ /dev/null @@ -1,5 +0,0 @@ - - -create external table nzhang_ctas4 as select key, value from src; - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_recursion.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_recursion.q deleted file mode 100644 index 2160b4719662b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_recursion.q +++ /dev/null @@ -1,4 +0,0 @@ -explain -with q1 as ( select key from q2 where key = '5'), -q2 as ( select key from q1 where key = '5') -select * from (select key from q1) a; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_with_in_subquery.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_with_in_subquery.q deleted file mode 100644 index e52a1d97db801..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_with_in_subquery.q +++ /dev/null @@ -1 +0,0 @@ -select * from (with q1 as ( select key from q2 where key = '5') select * from q1) a; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_create_already_exists.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_create_already_exists.q deleted file mode 100644 index 3af7607739a54..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_create_already_exists.q +++ /dev/null @@ -1,5 +0,0 @@ -SHOW DATABASES; - --- Try to create a database that already exists -CREATE DATABASE test_db; -CREATE DATABASE test_db; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_create_invalid_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_create_invalid_name.q deleted file mode 100644 index 5d6749542b470..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_create_invalid_name.q +++ /dev/null @@ -1,4 +0,0 @@ -SHOW DATABASES; - --- Try to create a database with an invalid name -CREATE DATABASE `test.db`; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q deleted file mode 100644 index 66a940e63dea6..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q +++ /dev/null @@ -1,4 +0,0 @@ -SHOW DATABASES; - --- Try to drop a database that does not exist -DROP DATABASE does_not_exist; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_not_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_not_empty.q deleted file mode 100644 index ae5a443f10627..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_not_empty.q +++ /dev/null @@ -1,8 +0,0 @@ -SHOW DATABASES; - --- Try to drop a non-empty database -CREATE DATABASE test_db; -USE test_db; -CREATE TABLE t(a INT); -USE default; -DROP DATABASE test_db; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_not_empty_restrict.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_not_empty_restrict.q deleted file mode 100644 index e1cb81c93f27b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_not_empty_restrict.q +++ /dev/null @@ -1,8 +0,0 @@ -SHOW DATABASES; - --- Try to drop a non-empty database in restrict mode -CREATE DATABASE db_drop_non_empty_restrict; -USE db_drop_non_empty_restrict; -CREATE TABLE t(a INT); -USE default; -DROP DATABASE db_drop_non_empty_restrict; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q deleted file mode 100644 index 5cd469769e0aa..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q +++ /dev/null @@ -1,4 +0,0 @@ -SHOW DATABASES; - --- Try to switch to a database that does not exist -USE does_not_exist; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal2.q deleted file mode 100644 index 711dc9e0fd357..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal2.q +++ /dev/null @@ -1,2 +0,0 @@ --- Not in YYYY-MM-DD format -SELECT DATE '2001/01/01' FROM src LIMIT 2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal3.q deleted file mode 100644 index 9483509b6bb79..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal3.q +++ /dev/null @@ -1,2 +0,0 @@ --- Invalid date value -SELECT DATE '2001-01-32' FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodblock.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodblock.q deleted file mode 100644 index 1c658c79b99ea..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodblock.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.support.concurrency=true; -set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; - -drop database if exists drop_nodblock; -create database drop_nodblock; -lock database drop_nodblock shared; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodbunlock.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodbunlock.q deleted file mode 100644 index ef4b323f063b6..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodbunlock.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.support.concurrency=true; -set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; - -drop database if exists drop_nodbunlock; -create database drop_nodbunlock; -unlock database drop_nodbunlock; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notablelock.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notablelock.q deleted file mode 100644 index 4a0c6c25c67c7..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notablelock.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.support.concurrency=true; -set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; - -drop table if exists drop_notablelock; -create table drop_notablelock (c int); -lock table drop_notablelock shared; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notableunlock.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notableunlock.q deleted file mode 100644 index 0b00046579f43..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notableunlock.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.support.concurrency=true; -set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; - -drop table if exists drop_notableunlock; -create table drop_notableunlock (c int); -unlock table drop_notableunlock; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ddltime.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ddltime.q deleted file mode 100644 index 3517a6046de14..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ddltime.q +++ /dev/null @@ -1,6 +0,0 @@ - -create table T2 like srcpart; - -insert overwrite table T2 partition (ds = '2010-06-21', hr='1') select /*+ HOLD_DDLTIME */ key, value from src where key > 10; - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/decimal_precision.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/decimal_precision.q deleted file mode 100644 index f49649837e214..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/decimal_precision.q +++ /dev/null @@ -1,10 +0,0 @@ -DROP TABLE IF EXISTS DECIMAL_PRECISION; - -CREATE TABLE DECIMAL_PRECISION(dec decimal) -ROW FORMAT DELIMITED - FIELDS TERMINATED BY ' ' -STORED AS TEXTFILE; - -SELECT dec * 123456789012345678901234567890.123456789bd FROM DECIMAL_PRECISION; - -DROP TABLE DECIMAL_PRECISION; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/decimal_precision_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/decimal_precision_1.q deleted file mode 100644 index 036ff1facc0a1..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/decimal_precision_1.q +++ /dev/null @@ -1,10 +0,0 @@ -DROP TABLE IF EXISTS DECIMAL_PRECISION; - -CREATE TABLE DECIMAL_PRECISION(dec decimal) -ROW FORMAT DELIMITED - FIELDS TERMINATED BY ' ' -STORED AS TEXTFILE; - -SELECT * from DECIMAL_PRECISION WHERE dec > 1234567890123456789.0123456789bd; - -DROP TABLE DECIMAL_PRECISION; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/default_partition_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/default_partition_name.q deleted file mode 100644 index 816b6cb80a964..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/default_partition_name.q +++ /dev/null @@ -1,3 +0,0 @@ -create table default_partition_name (key int, value string) partitioned by (ds string); - -alter table default_partition_name add partition(ds='__HIVE_DEFAULT_PARTITION__'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/deletejar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/deletejar.q deleted file mode 100644 index 0bd6985e031b6..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/deletejar.q +++ /dev/null @@ -1,4 +0,0 @@ - -ADD JAR ${system:maven.local.repository}/org/apache/hive/hive-it-test-serde/${system:hive.version}/hive-it-test-serde-${system:hive.version}.jar; -DELETE JAR ${system:maven.local.repository}/org/apache/hive/hive-it-test-serde/${system:hive.version}/hive-it-test-serde-${system:hive.version}.jar; -CREATE TABLE DELETEJAR(KEY STRING, VALUE STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure1.q deleted file mode 100644 index f7304b12e65f6..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure1.q +++ /dev/null @@ -1 +0,0 @@ -DESC NonExistentTable; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure2.q deleted file mode 100644 index f28b610466499..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure2.q +++ /dev/null @@ -1,2 +0,0 @@ -DESC srcpart; -DESC srcpart PARTITION(ds='2012-04-08', hr='15'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure3.q deleted file mode 100644 index bee0ea5788b4e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure3.q +++ /dev/null @@ -1,5 +0,0 @@ -CREATE DATABASE db1; -CREATE TABLE db1.t1(key1 INT, value1 STRING) PARTITIONED BY (ds STRING, part STRING); - --- describe database.table.column -DESCRIBE db1.t1.key1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath1.q deleted file mode 100644 index ea72f83e1d585..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath1.q +++ /dev/null @@ -1 +0,0 @@ -describe src_thrift.$elem$; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath2.q deleted file mode 100644 index f1fee1ac444de..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath2.q +++ /dev/null @@ -1 +0,0 @@ -describe src_thrift.$key$; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath3.q deleted file mode 100644 index 4a11f6845f396..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath3.q +++ /dev/null @@ -1 +0,0 @@ -describe src_thrift.lint.abc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath4.q deleted file mode 100644 index 0912bf1cd9ddd..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath4.q +++ /dev/null @@ -1 +0,0 @@ -describe src_thrift.mStringString.abc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on1.q deleted file mode 100644 index d0d748cf4ffd7..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on1.q +++ /dev/null @@ -1,17 +0,0 @@ -SET hive.metastore.disallow.incompatible.col.type.changes=true; -SELECT * FROM src LIMIT 1; -CREATE TABLE test_table123 (a INT, b MAP) PARTITIONED BY (ds STRING) STORED AS SEQUENCEFILE; -INSERT OVERWRITE TABLE test_table123 PARTITION(ds="foo1") SELECT 1, MAP("a1", "b1") FROM src LIMIT 1; -SELECT * from test_table123 WHERE ds="foo1"; -ALTER TABLE test_table123 REPLACE COLUMNS (a INT, b MAP); -ALTER TABLE test_table123 REPLACE COLUMNS (a BIGINT, b MAP); -ALTER TABLE test_table123 REPLACE COLUMNS (a INT, b MAP); -ALTER TABLE test_table123 REPLACE COLUMNS (a DOUBLE, b MAP); -ALTER TABLE test_table123 REPLACE COLUMNS (a TINYINT, b MAP); -ALTER TABLE test_table123 REPLACE COLUMNS (a BOOLEAN, b MAP); -ALTER TABLE test_table123 REPLACE COLUMNS (a TINYINT, b MAP); -ALTER TABLE test_table123 CHANGE COLUMN a a_new BOOLEAN; --- All the above ALTERs will succeed since they are between compatible types. --- The following ALTER will fail as MAP and STRING are not --- compatible. -ALTER TABLE test_table123 REPLACE COLUMNS (a INT, b STRING); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on2.q deleted file mode 100644 index 4460c3edd7e4b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on2.q +++ /dev/null @@ -1,6 +0,0 @@ -SET hive.metastore.disallow.incompatible.col.type.changes=true; -SELECT * FROM src LIMIT 1; -CREATE TABLE test_table123 (a INT, b STRING) PARTITIONED BY (ds STRING) STORED AS SEQUENCEFILE; -INSERT OVERWRITE TABLE test_table123 PARTITION(ds="foo1") SELECT 1, "one" FROM src LIMIT 1; -SELECT * from test_table123 WHERE ds="foo1"; -ALTER TABLE test_table123 CHANGE COLUMN b b MAP; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_func_nonexistent.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_func_nonexistent.q deleted file mode 100644 index 892ef00e3f86c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_func_nonexistent.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.exec.drop.ignorenonexistent=false; --- Can't use DROP FUNCTION if the function doesn't exist and IF EXISTS isn't specified -drop function nonexistent_function; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_function_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_function_failure.q deleted file mode 100644 index 51dc5e9d8e32c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_function_failure.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.exec.drop.ignorenonexistent=false; --- Can't use DROP TEMPORARY FUNCTION if the function doesn't exist and IF EXISTS isn't specified -DROP TEMPORARY FUNCTION UnknownFunction; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_index_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_index_failure.q deleted file mode 100644 index 6e907dfa99b24..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_index_failure.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.exec.drop.ignorenonexistent=false; --- Can't use DROP INDEX if the index doesn't exist and IF EXISTS isn't specified -DROP INDEX UnknownIndex ON src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_native_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_native_udf.q deleted file mode 100644 index ae047bbc1780f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_native_udf.q +++ /dev/null @@ -1 +0,0 @@ -DROP TEMPORARY FUNCTION max; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_failure.q deleted file mode 100644 index c2074f69cbf36..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_failure.q +++ /dev/null @@ -1,11 +0,0 @@ -create table mp (a string) partitioned by (b string, c string); - -alter table mp add partition (b='1', c='1'); -alter table mp add partition (b='1', c='2'); -alter table mp add partition (b='2', c='2'); - -show partitions mp; - -set hive.exec.drop.ignorenonexistent=false; --- Can't use DROP PARTITION if the partition doesn't exist and IF EXISTS isn't specified -alter table mp drop partition (b='3'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_filter_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_filter_failure.q deleted file mode 100644 index df476ed7c4634..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_filter_failure.q +++ /dev/null @@ -1,8 +0,0 @@ -create table ptestfilter1 (a string, b int) partitioned by (c string, d string); - -alter table ptestfilter1 add partition (c='US', d=1); -show partitions ptestfilter1; - -set hive.exec.drop.ignorenonexistent=false; -alter table ptestfilter1 drop partition (c='US', d<1); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure1.q deleted file mode 100644 index d47c08b876fca..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure1.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.exec.drop.ignorenonexistent=false; --- Can't use DROP TABLE if the table doesn't exist and IF EXISTS isn't specified -DROP TABLE UnknownTable; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure2.q deleted file mode 100644 index 631e4ffba7a42..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure2.q +++ /dev/null @@ -1,3 +0,0 @@ -CREATE VIEW xxx6 AS SELECT key FROM src; --- Can't use DROP TABLE on a view -DROP TABLE xxx6; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure3.q deleted file mode 100644 index 534ce0b0324af..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure3.q +++ /dev/null @@ -1,12 +0,0 @@ -create database dtf3; -use dtf3; - -create table drop_table_failure_temp(col STRING) partitioned by (p STRING); - -alter table drop_table_failure_temp add partition (p ='p1'); -alter table drop_table_failure_temp add partition (p ='p2'); -alter table drop_table_failure_temp add partition (p ='p3'); - -alter table drop_table_failure_temp partition (p ='p3') ENABLE NO_DROP; - -drop table drop_table_failure_temp; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_view_failure1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_view_failure1.q deleted file mode 100644 index 79cb4e445b05c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_view_failure1.q +++ /dev/null @@ -1,6 +0,0 @@ - - -CREATE TABLE xxx1(key int); - --- Can't use DROP VIEW on a base table -DROP VIEW xxx1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_view_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_view_failure2.q deleted file mode 100644 index 93bb16232d57d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_view_failure2.q +++ /dev/null @@ -1,3 +0,0 @@ -SET hive.exec.drop.ignorenonexistent=false; --- Can't use DROP VIEW if the view doesn't exist and IF EXISTS isn't specified -DROP VIEW UnknownView; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_alias_in_transform.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_alias_in_transform.q deleted file mode 100644 index b2e8567f09e16..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_alias_in_transform.q +++ /dev/null @@ -1 +0,0 @@ -FROM src SELECT TRANSFORM (key, value) USING "awk -F'\001' '{print $0}'" AS (foo, foo); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_alias_in_transform_schema.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_alias_in_transform_schema.q deleted file mode 100644 index dabbc351bc386..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_alias_in_transform_schema.q +++ /dev/null @@ -1 +0,0 @@ -FROM src SELECT TRANSFORM (key, value) USING "awk -F'\001' '{print $0}'" AS (foo STRING, foo STRING); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert1.q deleted file mode 100644 index fcbc7d5444a4d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert1.q +++ /dev/null @@ -1,7 +0,0 @@ - -create table dest1_din1(key int, value string); - -from src -insert overwrite table dest1_din1 select key, value -insert overwrite table dest1_din1 select key, value; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert2.q deleted file mode 100644 index 4f79a0352f21c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert2.q +++ /dev/null @@ -1,6 +0,0 @@ - -create table dest1_din2(key int, value string) partitioned by (ds string); - -from src -insert overwrite table dest1_din2 partition (ds='1') select key, value -insert overwrite table dest1_din2 partition (ds='1') select key, value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert3.q deleted file mode 100644 index 7b271a56d1844..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert3.q +++ /dev/null @@ -1,4 +0,0 @@ - -from src -insert overwrite directory '${system:test.tmp.dir}/dest1' select key, value -insert overwrite directory '${system:test.tmp.dir}/dest1' select key, value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part1.q deleted file mode 100644 index 9f0b6c7a0cc88..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part1.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.exec.dynamic.partition=true; -set hive.exec.dynamic.partition.mode=nostrict; -set hive.exec.max.dynamic.partitions=2; - - -create table dynamic_partition (key string) partitioned by (value string); - -insert overwrite table dynamic_partition partition(hr) select key, value from src; - - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part2.q deleted file mode 100644 index 00a92783c0548..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part2.q +++ /dev/null @@ -1,11 +0,0 @@ - -create table nzhang_part1 (key string, value string) partitioned by (ds string, hr string); - -set hive.exec.dynamic.partition=true; - -insert overwrite table nzhang_part1 partition(ds='11', hr) select key, value from srcpart where ds is not null; - -show partitions nzhang_part1; - - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part3.q deleted file mode 100644 index 7a8c58a6b255b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part3.q +++ /dev/null @@ -1,9 +0,0 @@ -set hive.exec.max.dynamic.partitions=600; -set hive.exec.max.dynamic.partitions.pernode=600; -set hive.exec.dynamic.partition.mode=nonstrict; -set hive.exec.dynamic.partition=true; -set hive.exec.max.created.files=100; - -create table nzhang_part( key string) partitioned by (value string); - -insert overwrite table nzhang_part partition(value) select key, value from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part4.q deleted file mode 100644 index 9aff7aa6310d2..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part4.q +++ /dev/null @@ -1,7 +0,0 @@ -create table nzhang_part4 (key string) partitioned by (ds string, hr string, value string); - -set hive.exec.dynamic.partition=true; - -insert overwrite table nzhang_part4 partition(value = 'aaa', ds='11', hr) select key, hr from srcpart where ds is not null; - -drop table nzhang_part4; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_empty.q.disabled b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_empty.q.disabled deleted file mode 100644 index a8fce595005d2..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_empty.q.disabled +++ /dev/null @@ -1,24 +0,0 @@ --- Licensed to the Apache Software Foundation (ASF) under one --- or more contributor license agreements. See the NOTICE file --- distributed with this work for additional information --- regarding copyright ownership. The ASF licenses this file --- to you under the Apache License, Version 2.0 (the --- "License"); you may not use this file except in compliance --- with the License. You may obtain a copy of the License at --- --- http://www.apache.org/licenses/LICENSE-2.0 --- --- Unless required by applicable law or agreed to in writing, software --- distributed under the License is distributed on an "AS IS" BASIS, --- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. --- See the License for the specific language governing permissions and --- limitations under the License. - -set hive.exec.dynamic.partition=true; -set hive.exec.dynamic.partition.mode=nonstrict; -set hive.stats.autogether=false; -set hive.error.on.empty.partition=true; - -create table dyn_err(key string, value string) partitioned by (ds string); - -insert overwrite table dyn_err partition(ds) select key, value, ds from srcpart where ds is not null and key = 'no exists'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_max.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_max.q deleted file mode 100644 index 6a7a6255b959b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_max.q +++ /dev/null @@ -1,16 +0,0 @@ -USE default; - --- Test of hive.exec.max.dynamic.partitions --- Set hive.exec.max.dynamic.partitions.pernode to a large value so it will be ignored - -CREATE TABLE max_parts(key STRING) PARTITIONED BY (value STRING); - -set hive.exec.dynamic.partition=true; -set hive.exec.dynamic.partition.mode=nonstrict; -set hive.exec.max.dynamic.partitions=10; -set hive.exec.max.dynamic.partitions.pernode=1000; - -INSERT OVERWRITE TABLE max_parts PARTITION(value) -SELECT key, value -FROM src -LIMIT 50; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_max_per_node.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_max_per_node.q deleted file mode 100644 index a411ec520b6d0..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_max_per_node.q +++ /dev/null @@ -1,15 +0,0 @@ -USE default; - --- Test of hive.exec.max.dynamic.partitions.pernode - -CREATE TABLE max_parts(key STRING) PARTITIONED BY (value STRING); - -set hive.exec.dynamic.partition=true; -set hive.exec.dynamic.partition.mode=nonstrict; -set hive.exec.max.dynamic.partitions=1000; -set hive.exec.max.dynamic.partitions.pernode=10; - -INSERT OVERWRITE TABLE max_parts PARTITION(value) -SELECT key, value -FROM src -LIMIT 50; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q deleted file mode 100644 index 0ad99d100dc07..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q +++ /dev/null @@ -1,17 +0,0 @@ -SET hive.metastore.partition.name.whitelist.pattern=[^9]*; -set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifyTableDirectoryIsEmptyHook; - -set hive.exec.dynamic.partition=true; -set hive.exec.dynamic.partition.mode=nonstrict; - -create table source_table like srcpart; - -create table dest_table like srcpart; - -load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE source_table partition(ds='2008-04-08', hr=11); - --- Tests creating dynamic partitions with characters not in the whitelist (i.e. 9) --- If the directory is not empty the hook will throw an error, instead the error should come from the metastore --- This shows that no dynamic partitions were created and left behind or had directories created - -insert overwrite table dest_table partition (ds, hr) select key, hr, ds, value from source_table where ds='2008-04-08' order by value asc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q deleted file mode 100644 index ca60d047efdd5..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q +++ /dev/null @@ -1,12 +0,0 @@ -CREATE TABLE exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING, hr STRING); -CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING, hr STRING); -SHOW PARTITIONS exchange_part_test1; -SHOW PARTITIONS exchange_part_test2; - -ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05', hr='h1'); -ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05', hr='h2'); -SHOW PARTITIONS exchange_part_test1; -SHOW PARTITIONS exchange_part_test2; - --- for exchange_part_test1 the value of ds is not given and the value of hr is given, thus this query will fail -alter table exchange_part_test1 exchange partition (hr='h1') with table exchange_part_test2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q deleted file mode 100644 index 7083edc32b98d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q +++ /dev/null @@ -1,12 +0,0 @@ -CREATE TABLE exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING); -CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING); -SHOW PARTITIONS exchange_part_test1; -SHOW PARTITIONS exchange_part_test2; - -ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05'); -ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05'); -SHOW PARTITIONS exchange_part_test1; -SHOW PARTITIONS exchange_part_test2; - --- exchange_part_test1 table partition (ds='2013-04-05') already exists thus this query will fail -alter table exchange_part_test1 exchange partition (ds='2013-04-05') with table exchange_part_test2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q deleted file mode 100644 index 6dfe81a8b0568..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q +++ /dev/null @@ -1,13 +0,0 @@ -CREATE TABLE exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING, hr STRING); -CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING, hr STRING); -SHOW PARTITIONS exchange_part_test1; -SHOW PARTITIONS exchange_part_test2; - -ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05', hr='1'); -ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05', hr='2'); -ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05', hr='3'); -SHOW PARTITIONS exchange_part_test1; -SHOW PARTITIONS exchange_part_test2; - --- exchange_part_test1 table partition (ds='2013-04-05') already exists thus this query will fail -alter table exchange_part_test1 exchange partition (ds='2013-04-05') with table exchange_part_test2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q deleted file mode 100644 index 60671e52e05d5..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q +++ /dev/null @@ -1,13 +0,0 @@ -CREATE TABLE exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING, hr STRING); -CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING, hr STRING); -SHOW PARTITIONS exchange_part_test1; -SHOW PARTITIONS exchange_part_test2; - -ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05', hr='1'); -ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05', hr='2'); -ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05', hr='1'); -SHOW PARTITIONS exchange_part_test1; -SHOW PARTITIONS exchange_part_test2; - --- exchange_part_test2 table partition (ds='2013-04-05') already exists thus this query will fail -alter table exchange_part_test1 exchange partition (ds='2013-04-05') with table exchange_part_test2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q deleted file mode 100644 index 38c0eda2368bd..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q +++ /dev/null @@ -1,6 +0,0 @@ -CREATE TABLE exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING); -CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING); -SHOW PARTITIONS exchange_part_test1; - --- exchange_part_test2 partition (ds='2013-04-05') does not exist thus this query will fail -alter table exchange_part_test1 exchange partition (ds='2013-04-05') with table exchange_part_test2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing.q deleted file mode 100644 index 7b926a3a8a51c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing.q +++ /dev/null @@ -1,2 +0,0 @@ --- t1 does not exist and the query fails -alter table t1 exchange partition (ds='2013-04-05') with table t2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing2.q deleted file mode 100644 index 48fcd74a6f22d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing2.q +++ /dev/null @@ -1,8 +0,0 @@ -CREATE TABLE exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING); -SHOW PARTITIONS exchange_part_test1; - -ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05'); -SHOW PARTITIONS exchange_part_test1; - --- exchange_part_test2 table does not exist thus this query will fail -alter table exchange_part_test1 exchange partition (ds='2013-04-05') with table exchange_part_test2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_test.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_test.q deleted file mode 100644 index 23e86e96ca4bc..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_test.q +++ /dev/null @@ -1,11 +0,0 @@ -CREATE TABLE exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING); -CREATE TABLE exchange_part_test2 (f1 string, f2 string) PARTITIONED BY (ds STRING); -SHOW PARTITIONS exchange_part_test1; -SHOW PARTITIONS exchange_part_test2; - -ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05'); -SHOW PARTITIONS exchange_part_test1; -SHOW PARTITIONS exchange_part_test2; - --- exchange_part_test1 and exchange_part_test2 do not have the same scheme and thus they fail -ALTER TABLE exchange_part_test1 EXCHANGE PARTITION (ds='2013-04-05') WITH TABLE exchange_part_test2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q deleted file mode 100644 index 6ffc33acb92ec..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q +++ /dev/null @@ -1,12 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'nosuchschema://nosuchauthority/ql/test/data/exports/exim_department'; -drop table exim_department; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q deleted file mode 100644 index 970e6463e24a5..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q +++ /dev/null @@ -1,24 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department identifier") - stored as textfile - tblproperties("maker"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q deleted file mode 100644 index 358918363d830..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q +++ /dev/null @@ -1,38 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_employee ( emp_id int comment "employee id") - comment "employee table" - partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="in", emp_state="ka"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="us", emp_state="tn"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_employee; -export table exim_employee to 'ql/test/data/exports/exim_employee'; -drop table exim_employee; - -create database importer; -use importer; - -create table exim_employee ( emp_id int comment "employee id") - comment "table of employees" - partitioned by (emp_country string comment "iso code", emp_state string comment "free-form text") - stored as textfile - tblproperties("maker"="krishna"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="us", emp_state="ka"); -import from 'ql/test/data/exports/exim_employee'; -describe extended exim_employee; -select * from exim_employee; -drop table exim_employee; -dfs -rmr target/tmp/ql/test/data/exports/exim_employee; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q deleted file mode 100644 index 45268c21c00e3..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q +++ /dev/null @@ -1,23 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_key int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q deleted file mode 100644 index cad6c90fd316e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q +++ /dev/null @@ -1,23 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department id", dep_name string) - stored as textfile - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q deleted file mode 100644 index f5f904f42af5a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q +++ /dev/null @@ -1,23 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id bigint comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q deleted file mode 100644 index c56329c03f89f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q +++ /dev/null @@ -1,23 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department id") - stored as rcfile - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q deleted file mode 100644 index afaedcd37bf72..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q +++ /dev/null @@ -1,26 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department id") - stored as inputformat "org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat" - outputformat "org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat" - inputdriver "org.apache.hadoop.hive.howl.rcfile.RCFileInputDriver" - outputdriver "org.apache.hadoop.hive.howl.rcfile.RCFileOutputDriver" - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q deleted file mode 100644 index 230b28c402cc5..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q +++ /dev/null @@ -1,24 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department id") - row format serde "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" - stored as textfile - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q deleted file mode 100644 index c2e00a9663468..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q +++ /dev/null @@ -1,28 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department id") - row format serde "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe" - with serdeproperties ("serialization.format"="0") - stored as inputformat "org.apache.hadoop.mapred.TextInputFormat" - outputformat "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat" - inputdriver "org.apache.hadoop.hive.howl.rcfile.RCFileInputDriver" - outputdriver "org.apache.hadoop.hive.howl.rcfile.RCFileOutputDriver" - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q deleted file mode 100644 index a6586ead0c23f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q +++ /dev/null @@ -1,24 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department id") - clustered by (dep_id) into 10 buckets - stored as textfile - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q deleted file mode 100644 index 990a686ebeea6..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q +++ /dev/null @@ -1,25 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - clustered by (dep_id) sorted by (dep_id desc) into 10 buckets - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department id") - clustered by (dep_id) sorted by (dep_id asc) into 10 buckets - stored as textfile - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_12_nonnative_export.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_12_nonnative_export.q deleted file mode 100644 index 289bcf001fded..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_12_nonnative_export.q +++ /dev/null @@ -1,9 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - clustered by (dep_id) sorted by (dep_id desc) into 10 buckets - stored by "org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler" - tblproperties("creator"="krishna"); -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q deleted file mode 100644 index 02537ef022d82..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q +++ /dev/null @@ -1,24 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department id") - stored by "org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler" - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; - \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q deleted file mode 100644 index 897c6747354b7..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q +++ /dev/null @@ -1,25 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department id") - partitioned by (dep_org string) - stored as textfile - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; - \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q deleted file mode 100644 index 12013e5ccfc49..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q +++ /dev/null @@ -1,25 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - partitioned by (dep_org string) - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department partition (dep_org="hr"); -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; - \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q deleted file mode 100644 index d8d2b8008c9ee..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q +++ /dev/null @@ -1,26 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - partitioned by (dep_org string) - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department partition (dep_org="hr"); -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department id") - partitioned by (dep_mgr string) - stored as textfile - tblproperties("creator"="krishna"); -import from 'ql/test/data/exports/exim_department'; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - -drop database importer; - \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q deleted file mode 100644 index 82dcce9455958..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q +++ /dev/null @@ -1,30 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_employee ( emp_id int comment "employee id") - comment "employee table" - partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="in", emp_state="ka"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="us", emp_state="tn"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_employee; -export table exim_employee to 'ql/test/data/exports/exim_employee'; -drop table exim_employee; - -create database importer; -use importer; -import table exim_employee partition (emp_country="us") from 'ql/test/data/exports/exim_employee'; -describe extended exim_employee; -select * from exim_employee; -drop table exim_employee; -dfs -rmr target/tmp/ql/test/data/exports/exim_employee; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q deleted file mode 100644 index d92efeb9a70ef..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q +++ /dev/null @@ -1,30 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_employee ( emp_id int comment "employee id") - comment "employee table" - partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="in", emp_state="ka"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="us", emp_state="tn"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_employee; -export table exim_employee to 'ql/test/data/exports/exim_employee'; -drop table exim_employee; - -create database importer; -use importer; -import table exim_employee partition (emp_country="us", emp_state="kl") from 'ql/test/data/exports/exim_employee'; -describe extended exim_employee; -select * from exim_employee; -drop table exim_employee; -dfs -rmr target/tmp/ql/test/data/exports/exim_employee; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q deleted file mode 100644 index 12d827b9c838c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q +++ /dev/null @@ -1,23 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -import external table exim_department from 'ql/test/data/exports/exim_department'; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -drop table exim_department; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q deleted file mode 100644 index 726dee53955af..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q +++ /dev/null @@ -1,30 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/tablestore/exim_department; - -create table exim_department ( dep_id int comment "department id") - stored as textfile - location 'ql/test/data/tablestore/exim_department' - tblproperties("creator"="krishna"); -import table exim_department from 'ql/test/data/exports/exim_department' - location 'ql/test/data/tablestore2/exim_department'; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -drop table exim_department; -dfs -rmr target/tmp/ql/test/data/tablestore/exim_department; - - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q deleted file mode 100644 index d187c78202034..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q +++ /dev/null @@ -1,35 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_employee ( emp_id int comment "employee id") - comment "employee table" - partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="in", emp_state="ka"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="us", emp_state="tn"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_employee; -export table exim_employee to 'ql/test/data/exports/exim_employee'; -drop table exim_employee; - -create database importer; -use importer; - -create table exim_employee ( emp_id int comment "employee id") - comment "employee table" - partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") - stored as textfile - tblproperties("creator"="krishna"); -import external table exim_employee partition (emp_country="us", emp_state="tn") - from 'ql/test/data/exports/exim_employee'; -dfs -rmr target/tmp/ql/test/data/exports/exim_employee; -drop table exim_employee; - -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_22_export_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_22_export_authfail.q deleted file mode 100644 index b818686f773df..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_22_export_authfail.q +++ /dev/null @@ -1,14 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int) stored as textfile; - -set hive.security.authorization.enabled=true; - -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; - -set hive.security.authorization.enabled=false; -drop table exim_department; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q deleted file mode 100644 index 4acefb9f0ae12..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q +++ /dev/null @@ -1,22 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; - -create table exim_department ( dep_id int) stored as textfile; -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -create table exim_department ( dep_id int) stored as textfile; -set hive.security.authorization.enabled=true; -import from 'ql/test/data/exports/exim_department'; - -set hive.security.authorization.enabled=false; -drop table exim_department; -drop database importer; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q deleted file mode 100644 index 467014e4679f6..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q +++ /dev/null @@ -1,31 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; -set hive.test.mode.nosamplelist=exim_department,exim_employee; - -create table exim_employee ( emp_id int comment "employee id") - comment "employee table" - partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") - stored as textfile - tblproperties("creator"="krishna"); -load data local inpath "../../data/files/test.dat" - into table exim_employee partition (emp_country="in", emp_state="tn"); -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_employee; -export table exim_employee to 'ql/test/data/exports/exim_employee'; -drop table exim_employee; - -create database importer; -use importer; -create table exim_employee ( emp_id int comment "employee id") - comment "employee table" - partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") - stored as textfile - tblproperties("creator"="krishna"); - -set hive.security.authorization.enabled=true; -import from 'ql/test/data/exports/exim_employee'; -set hive.security.authorization.enabled=false; - -dfs -rmr target/tmp/ql/test/data/exports/exim_employee; -drop table exim_employee; -drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q deleted file mode 100644 index 595fa7e764952..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q +++ /dev/null @@ -1,23 +0,0 @@ -set hive.test.mode=true; -set hive.test.mode.prefix=; -set hive.test.mode.nosamplelist=exim_department,exim_employee; - -create table exim_department ( dep_id int) stored as textfile; -load data local inpath "../../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; -export table exim_department to 'ql/test/data/exports/exim_department'; -drop table exim_department; - -create database importer; -use importer; - -set hive.security.authorization.enabled=true; -import from 'ql/test/data/exports/exim_department'; - -set hive.security.authorization.enabled=false; -select * from exim_department; -drop table exim_department; -drop database importer; -dfs -rmr target/tmp/ql/test/data/exports/exim_department; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/external1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/external1.q deleted file mode 100644 index d56c955050bc5..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/external1.q +++ /dev/null @@ -1,3 +0,0 @@ - -create external table external1(a int, b int) location 'invalidscheme://data.s3ndemo.hive/kv'; -describe external1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/external2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/external2.q deleted file mode 100644 index 0df85a09afdd5..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/external2.q +++ /dev/null @@ -1,4 +0,0 @@ - -create external table external2(a int, b int) partitioned by (ds string); -alter table external2 add partition (ds='2008-01-01') location 'invalidscheme://data.s3ndemo.hive/pkv/2008-01-01'; -describe external2 partition (ds='2008-01-01'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fetchtask_ioexception.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fetchtask_ioexception.q deleted file mode 100644 index 82230f782eac3..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fetchtask_ioexception.q +++ /dev/null @@ -1,7 +0,0 @@ -CREATE TABLE fetchtask_ioexception ( - KEY STRING, - VALUE STRING) STORED AS SEQUENCEFILE; - -LOAD DATA LOCAL INPATH '../../data/files/kv1_broken.seq' OVERWRITE INTO TABLE fetchtask_ioexception; - -SELECT * FROM fetchtask_ioexception; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/file_with_header_footer_negative.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/file_with_header_footer_negative.q deleted file mode 100644 index 286cf1afb491b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/file_with_header_footer_negative.q +++ /dev/null @@ -1,13 +0,0 @@ -dfs ${system:test.dfs.mkdir} hdfs:///tmp/test_file_with_header_footer_negative/; - -dfs -copyFromLocal ../data/files/header_footer_table_1 hdfs:///tmp/test_file_with_header_footer_negative/header_footer_table_1; - -dfs -copyFromLocal ../data/files/header_footer_table_2 hdfs:///tmp/test_file_with_header_footer_negative/header_footer_table_2; - -CREATE EXTERNAL TABLE header_footer_table_1 (name string, message string, id int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LOCATION 'hdfs:///tmp/test_file_with_header_footer_negative/header_footer_table_1' tblproperties ("skip.header.line.count"="1", "skip.footer.line.count"="200"); - -SELECT * FROM header_footer_table_1; - -DROP TABLE header_footer_table_1; - -dfs -rmr hdfs:///tmp/test_file_with_header_footer_negative; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_bad_class.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_bad_class.q deleted file mode 100644 index 33dd4fa614f05..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_bad_class.q +++ /dev/null @@ -1,3 +0,0 @@ -CREATE TABLE dest1(key INT, value STRING) STORED AS - INPUTFORMAT 'ClassDoesNotExist' - OUTPUTFORMAT 'java.lang.Void'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_void_input.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_void_input.q deleted file mode 100644 index c514562b24160..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_void_input.q +++ /dev/null @@ -1,8 +0,0 @@ -CREATE TABLE dest1(key INT, value STRING) STORED AS - INPUTFORMAT 'java.lang.Void' - OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat'; - -FROM src -INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value WHERE src.key < 10; - -SELECT dest1.* FROM dest1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_void_output.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_void_output.q deleted file mode 100644 index a9cef1eada16a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_void_output.q +++ /dev/null @@ -1,6 +0,0 @@ -CREATE TABLE dest1(key INT, value STRING) STORED AS - INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat' - OUTPUTFORMAT 'java.lang.Void'; - -FROM src -INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value WHERE src.key < 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fs_default_name1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fs_default_name1.q deleted file mode 100644 index f50369b138573..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fs_default_name1.q +++ /dev/null @@ -1,2 +0,0 @@ -set fs.default.name='http://www.example.com; -show tables; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fs_default_name2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fs_default_name2.q deleted file mode 100644 index 485c3db06823f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fs_default_name2.q +++ /dev/null @@ -1,2 +0,0 @@ -set fs.default.name='http://www.example.com; -SELECT * FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/genericFileFormat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/genericFileFormat.q deleted file mode 100644 index bd633b9760ab6..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/genericFileFormat.q +++ /dev/null @@ -1 +0,0 @@ -create table testFail (a int) stored as foo; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby2_map_skew_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby2_map_skew_multi_distinct.q deleted file mode 100644 index cecd9c6bd807c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby2_map_skew_multi_distinct.q +++ /dev/null @@ -1,14 +0,0 @@ -set hive.map.aggr=true; -set hive.groupby.skewindata=true; -set mapred.reduce.tasks=31; - -CREATE TABLE dest1(key STRING, c1 INT, c2 STRING, c3 INT, c4 INT) STORED AS TEXTFILE; - -EXPLAIN -FROM src -INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(DISTINCT substr(src.value, 5)), count(src.value) GROUP BY substr(src.key,1,1); - -FROM src -INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(DISTINCT substr(src.value, 5)), count(src.value) GROUP BY substr(src.key,1,1); - -SELECT dest1.* FROM dest1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby2_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby2_multi_distinct.q deleted file mode 100644 index e3b0066112c5c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby2_multi_distinct.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.map.aggr=false; -set hive.groupby.skewindata=true; - -CREATE TABLE dest_g2(key STRING, c1 INT, c2 STRING, c3 INT, c4 INT) STORED AS TEXTFILE; - -EXPLAIN -FROM src -INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(DISTINCT substr(src.value, 5)), count(src.value) GROUP BY substr(src.key,1,1); - -FROM src -INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(DISTINCT substr(src.value, 5)), count(src.value) GROUP BY substr(src.key,1,1); - -SELECT dest_g2.* FROM dest_g2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby3_map_skew_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby3_map_skew_multi_distinct.q deleted file mode 100644 index 168aeb1261b33..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby3_map_skew_multi_distinct.q +++ /dev/null @@ -1,36 +0,0 @@ -set hive.map.aggr=true; -set hive.groupby.skewindata=true; -set mapred.reduce.tasks=31; - -CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE, c10 DOUBLE, c11 DOUBLE) STORED AS TEXTFILE; - -EXPLAIN -FROM src -INSERT OVERWRITE TABLE dest1 SELECT - sum(substr(src.value,5)), - avg(substr(src.value,5)), - avg(DISTINCT substr(src.value,5)), - max(substr(src.value,5)), - min(substr(src.value,5)), - std(substr(src.value,5)), - stddev_samp(substr(src.value,5)), - variance(substr(src.value,5)), - var_samp(substr(src.value,5)), - sum(DISTINCT substr(src.value, 5)), - count(DISTINCT substr(src.value, 5)); - -FROM src -INSERT OVERWRITE TABLE dest1 SELECT - sum(substr(src.value,5)), - avg(substr(src.value,5)), - avg(DISTINCT substr(src.value,5)), - max(substr(src.value,5)), - min(substr(src.value,5)), - std(substr(src.value,5)), - stddev_samp(substr(src.value,5)), - variance(substr(src.value,5)), - var_samp(substr(src.value,5)), - sum(DISTINCT substr(src.value, 5)), - count(DISTINCT substr(src.value, 5)); - -SELECT dest1.* FROM dest1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby3_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby3_multi_distinct.q deleted file mode 100644 index 1a28477918c8e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby3_multi_distinct.q +++ /dev/null @@ -1,36 +0,0 @@ -set hive.map.aggr=false; -set hive.groupby.skewindata=true; - -CREATE TABLE dest1(c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE, c10 DOUBLE, c11 DOUBLE) STORED AS TEXTFILE; - -EXPLAIN -FROM src -INSERT OVERWRITE TABLE dest1 SELECT - sum(substr(src.value,5)), - avg(substr(src.value,5)), - avg(DISTINCT substr(src.value,5)), - max(substr(src.value,5)), - min(substr(src.value,5)), - std(substr(src.value,5)), - stddev_samp(substr(src.value,5)), - variance(substr(src.value,5)), - var_samp(substr(src.value,5)), - sum(DISTINCT substr(src.value, 5)), - count(DISTINCT substr(src.value, 5)); - - -FROM src -INSERT OVERWRITE TABLE dest1 SELECT - sum(substr(src.value,5)), - avg(substr(src.value,5)), - avg(DISTINCT substr(src.value,5)), - max(substr(src.value,5)), - min(substr(src.value,5)), - std(substr(src.value,5)), - stddev_samp(substr(src.value,5)), - variance(substr(src.value,5)), - var_samp(substr(src.value,5)), - sum(DISTINCT substr(src.value, 5)), - count(DISTINCT substr(src.value, 5)); - -SELECT dest1.* FROM dest1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_cube1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_cube1.q deleted file mode 100644 index a0bc177ad6351..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_cube1.q +++ /dev/null @@ -1,4 +0,0 @@ -set hive.map.aggr=false; - -SELECT key, count(distinct value) FROM src GROUP BY key with cube; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_cube2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_cube2.q deleted file mode 100644 index f8ecb6a2d4347..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_cube2.q +++ /dev/null @@ -1,4 +0,0 @@ -set hive.map.aggr=true; - -SELECT key, value, count(distinct value) FROM src GROUP BY key, value with cube; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_id1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_id1.q deleted file mode 100644 index ac5b6f7b03056..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_id1.q +++ /dev/null @@ -1,4 +0,0 @@ -CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE; - -SELECT GROUPING__ID FROM T1; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets1.q deleted file mode 100644 index ec6b16bfb28c6..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets1.q +++ /dev/null @@ -1,5 +0,0 @@ -CREATE TABLE T1(a STRING, b STRING, c STRING); - --- Check for empty grouping set -SELECT * FROM T1 GROUP BY a GROUPING SETS (()); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets2.q deleted file mode 100644 index c988e04e74fa2..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets2.q +++ /dev/null @@ -1,4 +0,0 @@ -CREATE TABLE T1(a STRING, b STRING, c STRING); - --- Check for mupltiple empty grouping sets -SELECT * FROM T1 GROUP BY b GROUPING SETS ((), (), ()); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets3.q deleted file mode 100644 index 3e73552422956..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets3.q +++ /dev/null @@ -1,4 +0,0 @@ -CREATE TABLE T1(a STRING, b STRING, c STRING); - --- Grouping sets expression is not in GROUP BY clause -SELECT a FROM T1 GROUP BY a GROUPING SETS (a, b); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets4.q deleted file mode 100644 index cf6352c47d7eb..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets4.q +++ /dev/null @@ -1,4 +0,0 @@ -CREATE TABLE T1(a STRING, b STRING, c STRING); - --- Expression 'a' is not in GROUP BY clause -SELECT a FROM T1 GROUP BY b GROUPING SETS (b); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets5.q deleted file mode 100644 index 7df3318a644c9..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets5.q +++ /dev/null @@ -1,5 +0,0 @@ -CREATE TABLE T1(a STRING, b STRING, c STRING); - --- Alias in GROUPING SETS -SELECT a as c, count(*) FROM T1 GROUP BY c GROUPING SETS (c); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets6.q deleted file mode 100644 index 2783047698e78..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets6.q +++ /dev/null @@ -1,8 +0,0 @@ -set hive.new.job.grouping.set.cardinality=2; - -CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; - --- Since 4 grouping sets would be generated for the query below, an additional MR job should be created --- This is not allowed with distincts. -SELECT a, b, count(distinct c) from T1 group by a, b with cube; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets7.q deleted file mode 100644 index 6c9d5133ad7ee..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets7.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.new.job.grouping.set.cardinality=2; -set hive.map.aggr=true; -set hive.groupby.skewindata=true; - -CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; - --- Since 4 grouping sets would be generated for the query below, an additional MR job should be created --- This is not allowed with map-side aggregation and skew -SELECT a, b, count(1) from T1 group by a, b with cube; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_invalid_position.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_invalid_position.q deleted file mode 100644 index 173a752e351a8..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_invalid_position.q +++ /dev/null @@ -1,4 +0,0 @@ -set hive.groupby.orderby.position.alias=true; - --- invalid position alias in group by -SELECT src.key, sum(substr(src.value,5)) FROM src GROUP BY 3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_key.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_key.q deleted file mode 100644 index 20970152c33cc..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_key.q +++ /dev/null @@ -1 +0,0 @@ -SELECT concat(value, concat(value)) FROM src GROUP BY concat(value); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_rollup1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_rollup1.q deleted file mode 100644 index 6366744276077..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_rollup1.q +++ /dev/null @@ -1,4 +0,0 @@ -set hive.map.aggr=false; - -SELECT key, value, count(1) FROM src GROUP BY key, value with rollup; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_rollup2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_rollup2.q deleted file mode 100644 index aa19b523e9d92..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_rollup2.q +++ /dev/null @@ -1,4 +0,0 @@ -set hive.map.aggr=true; - -SELECT key, value, count(key) FROM src GROUP BY key, value with rollup; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/having1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/having1.q deleted file mode 100644 index 71f4fd13a0a0e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/having1.q +++ /dev/null @@ -1,2 +0,0 @@ -EXPLAIN SELECT * FROM src HAVING key > 300; -SELECT * FROM src HAVING key > 300; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type.q deleted file mode 100644 index 1ab828c8beae4..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type.q +++ /dev/null @@ -1,7 +0,0 @@ --- begin part(string, int) pass(string, string) -CREATE TABLE tab1 (id1 int,id2 string) PARTITIONED BY(month string,day int) ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' ; -LOAD DATA LOCAL INPATH '../../data/files/T1.txt' overwrite into table tab1 PARTITION(month='June', day='second'); - -select * from tab1; -drop table tab1; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type2.q deleted file mode 100644 index 2438288209896..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type2.q +++ /dev/null @@ -1,3 +0,0 @@ -create table tab1 (id1 int, id2 string) PARTITIONED BY(month string,day int) row format delimited fields terminated by ','; -alter table tab1 add partition (month='June', day='second'); -drop table tab1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type3.q deleted file mode 100644 index 49e6a092fc127..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type3.q +++ /dev/null @@ -1,4 +0,0 @@ -create table tab1(c int) partitioned by (i int); -alter table tab1 add partition(i = "some name"); - -drop table tab1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type4.q deleted file mode 100644 index 50f486e6245cf..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type4.q +++ /dev/null @@ -1,3 +0,0 @@ -create table tab1(s string) PARTITIONED BY(dt date, st string); -alter table tab1 add partition (dt=date 'foo', st='foo'); -drop table tab1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_bitmap_no_map_aggr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_bitmap_no_map_aggr.q deleted file mode 100644 index a17cd1fec5366..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_bitmap_no_map_aggr.q +++ /dev/null @@ -1,7 +0,0 @@ -EXPLAIN -CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; - -SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; -SET hive.map.aggr=false; -CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; -ALTER INDEX src1_index ON src REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_entry_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_entry_limit.q deleted file mode 100644 index 5bb889c027743..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_entry_limit.q +++ /dev/null @@ -1,12 +0,0 @@ -set hive.stats.dbclass=fs; -drop index src_index on src; - -CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; -ALTER INDEX src_index ON src REBUILD; - -SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; -INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_result" SELECT `_bucketname` , `_offsets` FROM default__src_src_index__ WHERE key<1000; -SET hive.index.compact.file=${system:test.tmp.dir}/index_result; -SET hive.input.format=org.apache.hadoop.hive.ql.index.compact.HiveCompactIndexInputFormat; -SET hive.index.compact.query.max.entries=5; -SELECT key, value FROM src WHERE key=100 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_size_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_size_limit.q deleted file mode 100644 index c6600e69b6a7c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_size_limit.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.stats.dbclass=fs; -drop index src_index on src; - -CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; -ALTER INDEX src_index ON src REBUILD; - -SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; -INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_result" SELECT `_bucketname` , `_offsets` FROM default__src_src_index__ WHERE key<1000; -SET hive.index.compact.file=${system:test.tmp.dir}/index_result; -SET hive.input.format=org.apache.hadoop.hive.ql.index.compact.HiveCompactIndexInputFormat; -SET hive.index.compact.query.max.size=1024; -SELECT key, value FROM src WHERE key=100 ORDER BY key; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input1.q deleted file mode 100644 index 92a6791acb652..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input1.q +++ /dev/null @@ -1 +0,0 @@ -SELECT a.* FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input2.q deleted file mode 100644 index 0fe907d9d8ae7..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input2.q +++ /dev/null @@ -1 +0,0 @@ -SELECT a.key FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input4.q deleted file mode 100644 index 60aea3208c4ed..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input4.q +++ /dev/null @@ -1,5 +0,0 @@ -set hive.mapred.mode=strict; - -select * from srcpart a join - (select b.key, count(1) as count from srcpart b where b.ds = '2008-04-08' and b.hr = '14' group by b.key) subq - where a.ds = '2008-04-08' and a.hr = '11' limit 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input41.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input41.q deleted file mode 100644 index 872ab1014874d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input41.q +++ /dev/null @@ -1,5 +0,0 @@ -select * from - (select * from src - union all - select * from srcpart where ds = '2009-08-09' - )x; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input_part0_neg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input_part0_neg.q deleted file mode 100644 index 4656693d4838c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input_part0_neg.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.mapred.mode=strict; - -SELECT x.* FROM SRCPART x WHERE key = '2008-04-08'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into1.q deleted file mode 100644 index 8c197670211bf..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into1.q +++ /dev/null @@ -1,11 +0,0 @@ -set hive.lock.numretries=5; -set hive.lock.sleep.between.retries=5; - -DROP TABLE insert_into1_neg; - -CREATE TABLE insert_into1_neg (key int, value string); - -LOCK TABLE insert_into1_neg SHARED; -INSERT INTO TABLE insert_into1_neg SELECT * FROM src LIMIT 100; - -DROP TABLE insert_into1_neg; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into2.q deleted file mode 100644 index 73a3b6ff13705..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into2.q +++ /dev/null @@ -1,10 +0,0 @@ -set hive.lock.numretries=5; -set hive.lock.sleep.between.retries=5; - -DROP TABLE insert_into1_neg; -CREATE TABLE insert_into1_neg (key int, value string); - -LOCK TABLE insert_into1_neg EXCLUSIVE; -INSERT INTO TABLE insert_into1_neg SELECT * FROM src LIMIT 100; - -DROP TABLE insert_into1_neg; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into3.q deleted file mode 100644 index 4d048b337ec45..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into3.q +++ /dev/null @@ -1,16 +0,0 @@ -set hive.lock.numretries=5; -set hive.lock.sleep.between.retries=5; - -DROP TABLE insert_into3_neg; - -CREATE TABLE insert_into3_neg (key int, value string) - PARTITIONED BY (ds string); - -INSERT INTO TABLE insert_into3_neg PARTITION (ds='1') - SELECT * FROM src LIMIT 100; - -LOCK TABLE insert_into3_neg PARTITION (ds='1') SHARED; -INSERT INTO TABLE insert_into3_neg PARTITION (ds='1') - SELECT * FROM src LIMIT 100; - -DROP TABLE insert_into3_neg; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into4.q deleted file mode 100644 index b8944e742b4da..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into4.q +++ /dev/null @@ -1,16 +0,0 @@ -set hive.lock.numretries=5; -set hive.lock.sleep.between.retries=5; - -DROP TABLE insert_into3_neg; - -CREATE TABLE insert_into3_neg (key int, value string) - PARTITIONED BY (ds string); - -INSERT INTO TABLE insert_into3_neg PARTITION (ds='1') - SELECT * FROM src LIMIT 100; - -LOCK TABLE insert_into3_neg PARTITION (ds='1') EXCLUSIVE; -INSERT INTO TABLE insert_into3_neg PARTITION (ds='1') - SELECT * FROM src LIMIT 100; - -DROP TABLE insert_into3_neg; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into5.q deleted file mode 100644 index c20c168a887c2..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into5.q +++ /dev/null @@ -1,9 +0,0 @@ -DROP TABLE if exists insert_into5_neg; - -CREATE TABLE insert_into5_neg (key int, value string) TBLPROPERTIES ("immutable"="true"); - -INSERT INTO TABLE insert_into5_neg SELECT * FROM src LIMIT 100; - -INSERT INTO TABLE insert_into5_neg SELECT * FROM src LIMIT 100; - -DROP TABLE insert_into5_neg; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into6.q deleted file mode 100644 index a92ee5ca94a33..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into6.q +++ /dev/null @@ -1,17 +0,0 @@ -DROP TABLE IF EXISTS insert_into6_neg; - -CREATE TABLE insert_into6_neg (key int, value string) - PARTITIONED BY (ds string) TBLPROPERTIES("immutable"="true") ; - -INSERT INTO TABLE insert_into6_neg PARTITION (ds='1') - SELECT * FROM src LIMIT 100; - -INSERT INTO TABLE insert_into6_neg PARTITION (ds='2') - SELECT * FROM src LIMIT 100; - -SELECT COUNT(*) from insert_into6_neg; - -INSERT INTO TABLE insert_into6_neg PARTITION (ds='1') - SELECT * FROM src LIMIT 100; - -DROP TABLE insert_into6_neg; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_view_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_view_failure.q deleted file mode 100644 index 1f5e13906259a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_view_failure.q +++ /dev/null @@ -1,5 +0,0 @@ -DROP VIEW xxx2; -CREATE VIEW xxx2 AS SELECT * FROM src; -INSERT OVERWRITE TABLE xxx2 -SELECT key, value -FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertexternal1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertexternal1.q deleted file mode 100644 index 01ebae1022324..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertexternal1.q +++ /dev/null @@ -1,8 +0,0 @@ -set hive.insert.into.external.tables=false; - - -create external table texternal(key string, val string) partitioned by (insertdate string); - -alter table texternal add partition (insertdate='2008-01-01') location 'pfile://${system:test.tmp.dir}/texternal/2008-01-01'; -from src insert overwrite table texternal partition (insertdate='2008-01-01') select *; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q deleted file mode 100644 index a8f77c28a8251..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q +++ /dev/null @@ -1,9 +0,0 @@ -set hive.exec.dynamic.partition=true; - -create table srcpart_dp like srcpart; - -create table destpart_dp like srcpart; - -load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_dp partition(ds='2008-04-08', hr=11); - -insert overwrite table destpart_dp partition (ds='2008-04-08', hr) if not exists select key, value, hr from srcpart_dp where ds='2008-04-08'; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_arithmetic_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_arithmetic_type.q deleted file mode 100644 index ad37cff79b587..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_arithmetic_type.q +++ /dev/null @@ -1,3 +0,0 @@ - -select timestamp('2001-01-01 00:00:01') - timestamp('2000-01-01 00:00:01') from src; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_avg_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_avg_syntax.q deleted file mode 100644 index d5b58e0765536..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_avg_syntax.q +++ /dev/null @@ -1 +0,0 @@ -SELECT avg(*) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_1.q deleted file mode 100644 index 73e4729aa0fc1..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_1.q +++ /dev/null @@ -1,2 +0,0 @@ -create table tbl (a binary); -select cast (a as int) from tbl limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_2.q deleted file mode 100644 index 50ec48152548c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_2.q +++ /dev/null @@ -1,2 +0,0 @@ -create table tbl (a binary); -select cast (a as tinyint) from tbl limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_3.q deleted file mode 100644 index 16f56ec5d3403..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_3.q +++ /dev/null @@ -1,2 +0,0 @@ -create table tbl (a binary); -select cast (a as smallint) from tbl limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_4.q deleted file mode 100644 index bd222f14b469a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_4.q +++ /dev/null @@ -1,2 +0,0 @@ -create table tbl (a binary); -select cast (a as bigint) from tbl limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_5.q deleted file mode 100644 index 594fd2bb6f625..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_5.q +++ /dev/null @@ -1,2 +0,0 @@ -create table tbl (a binary); -select cast (a as float) from tbl limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_6.q deleted file mode 100644 index 40ff801460ef8..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_6.q +++ /dev/null @@ -1,2 +0,0 @@ -create table tbl (a binary); -select cast (a as double) from tbl limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_1.q deleted file mode 100644 index 00cd98ed13b7d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_1.q +++ /dev/null @@ -1 +0,0 @@ -select cast (2 as binary) from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_2.q deleted file mode 100644 index f31344f835bb2..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_2.q +++ /dev/null @@ -1 +0,0 @@ -select cast(cast (2 as smallint) as binary) from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_3.q deleted file mode 100644 index af23d29f4e984..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_3.q +++ /dev/null @@ -1 +0,0 @@ -select cast(cast (2 as tinyint) as binary) from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_4.q deleted file mode 100644 index 91abe1e6b8a27..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_4.q +++ /dev/null @@ -1 +0,0 @@ -select cast(cast (2 as bigint) as binary) from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_5.q deleted file mode 100644 index afd99be9765a3..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_5.q +++ /dev/null @@ -1 +0,0 @@ -select cast(cast (2 as float) as binary) from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_6.q deleted file mode 100644 index c2143c5c9e955..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_6.q +++ /dev/null @@ -1 +0,0 @@ -select cast(cast (2 as double) as binary) from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_1.q deleted file mode 100644 index ba7d164c77155..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_1.q +++ /dev/null @@ -1,2 +0,0 @@ -drop table invalid_char_length_1; -create table invalid_char_length_1 (c1 char(1000000)); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_2.q deleted file mode 100644 index 866b43d31273d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_2.q +++ /dev/null @@ -1 +0,0 @@ -select cast(value as char(100000)) from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_3.q deleted file mode 100644 index 481b630d20489..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_3.q +++ /dev/null @@ -1,3 +0,0 @@ -drop table invalid_char_length_3; -create table invalid_char_length_3 (c1 char(0)); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_config1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_config1.q deleted file mode 100644 index c49ac8a69086e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_config1.q +++ /dev/null @@ -1,3 +0,0 @@ -set mapred.input.dir.recursive=true; - -CREATE TABLE skewedtable (key STRING, value STRING) SKEWED BY (key) ON (1,5,6); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_config2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_config2.q deleted file mode 100644 index fa023c8c4b5f2..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_config2.q +++ /dev/null @@ -1,4 +0,0 @@ -set hive.mapred.supports.subdirectories=false; -set hive.optimize.union.remove=true; - -select count(1) from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_create_tbl1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_create_tbl1.q deleted file mode 100644 index 2e1ea6b005618..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_create_tbl1.q +++ /dev/null @@ -1,9 +0,0 @@ - -CREATE TABLE inv_valid_tbl1 COMMENT 'This is a thrift based table' - PARTITIONED BY(aint DATETIME, country STRING) - CLUSTERED BY(aint) SORTED BY(lint) INTO 32 BUCKETS - ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer' - WITH SERDEPROPERTIES ('serialization.class' = 'org.apache.hadoop.hive.serde2.thrift.test.Complex', - 'serialization.format' = 'org.apache.thrift.protocol.TBinaryProtocol') - STORED AS SEQUENCEFILE; -DESCRIBE EXTENDED inv_valid_tbl1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_create_tbl2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_create_tbl2.q deleted file mode 100644 index 408919ee2d633..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_create_tbl2.q +++ /dev/null @@ -1 +0,0 @@ -create tabl tmp_zshao_22 (id int, name strin; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_mapjoin1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_mapjoin1.q deleted file mode 100644 index 56d9211d28ebb..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_mapjoin1.q +++ /dev/null @@ -1 +0,0 @@ -select /*+ MAPJOIN(a) ,MAPJOIN(b)*/ * from src a join src b on (a.key=b.key and a.value=b.value); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_max_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_max_syntax.q deleted file mode 100644 index 20033734090f3..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_max_syntax.q +++ /dev/null @@ -1 +0,0 @@ -SELECT max(*) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_min_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_min_syntax.q deleted file mode 100644 index 584283a08a9e7..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_min_syntax.q +++ /dev/null @@ -1 +0,0 @@ -SELECT min(*) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column.q deleted file mode 100644 index 106ba42213197..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column.q +++ /dev/null @@ -1,4 +0,0 @@ --- Create table -create table if not exists test_invalid_column(key string, value string ) partitioned by (year string, month string) stored as textfile ; - -select * from test_invalid_column where column1=123; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column_with_subquery.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column_with_subquery.q deleted file mode 100644 index bc70dbca20772..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column_with_subquery.q +++ /dev/null @@ -1,4 +0,0 @@ --- Create table -create table if not exists test_invalid_column(key string, value string ) partitioned by (year string, month string) stored as textfile ; - -select * from (select * from test_invalid_column) subq where subq = 123; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column_with_tablename.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column_with_tablename.q deleted file mode 100644 index b821e6129a7bf..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column_with_tablename.q +++ /dev/null @@ -1,4 +0,0 @@ --- Create table -create table if not exists test_invalid_column(key string, value string ) partitioned by (year string, month string) stored as textfile ; - -select * from test_invalid_column where test_invalid_column=123; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_expression.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_expression.q deleted file mode 100644 index 01617f9363b58..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_expression.q +++ /dev/null @@ -1 +0,0 @@ -select foo from a a where foo > .foo; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_std_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_std_syntax.q deleted file mode 100644 index 13104198a6db0..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_std_syntax.q +++ /dev/null @@ -1 +0,0 @@ -SELECT std(*) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q deleted file mode 100644 index c6a12526559e0..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q +++ /dev/null @@ -1 +0,0 @@ -SELECT stddev_samp(*) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_sum_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_sum_syntax.q deleted file mode 100644 index 2d591baa24eb7..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_sum_syntax.q +++ /dev/null @@ -1 +0,0 @@ -SELECT sum(*) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_alter1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_alter1.q deleted file mode 100644 index bb19cff8a93ee..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_alter1.q +++ /dev/null @@ -1,2 +0,0 @@ -CREATE TABLE alter_test (d STRING); -ALTER TABLE alter_test CHANGE d d DATETIME; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_alter2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_alter2.q deleted file mode 100644 index aa01b358727b1..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_alter2.q +++ /dev/null @@ -1,2 +0,0 @@ -CREATE TABLE alter_test (d STRING); -ALTER TABLE alter_test ADD COLUMNS (ds DATETIME); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_create2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_create2.q deleted file mode 100644 index 978f4244a6ba0..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_create2.q +++ /dev/null @@ -1 +0,0 @@ -CREATE TABLE datetime_test (d DATETIME); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_transform.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_transform.q deleted file mode 100644 index dfc4864acf43f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_transform.q +++ /dev/null @@ -1 +0,0 @@ -SELECT TRANSFORM(*) USING 'cat' AS (key DATETIME) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_tbl_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_tbl_name.q deleted file mode 100644 index 09394e71ada96..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_tbl_name.q +++ /dev/null @@ -1 +0,0 @@ -create table invalid-name(a int, b string); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q deleted file mode 100644 index ce2a8c4769111..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q +++ /dev/null @@ -1 +0,0 @@ -SELECT var_samp(*) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_1.q deleted file mode 100644 index 43de018c9f14c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_1.q +++ /dev/null @@ -1,2 +0,0 @@ -drop table if exists invalid_varchar_length_1; -create table invalid_varchar_length_1 (c1 varchar(1000000)); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_2.q deleted file mode 100644 index 3c199d31e7ffc..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_2.q +++ /dev/null @@ -1 +0,0 @@ -select cast(value as varchar(100000)) from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_3.q deleted file mode 100644 index fed04764a9440..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_3.q +++ /dev/null @@ -1,3 +0,0 @@ -drop table if exists invalid_varchar_length_3; -create table invalid_varchar_length_3 (c1 varchar(0)); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_variance_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_variance_syntax.q deleted file mode 100644 index 5b478299317aa..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_variance_syntax.q +++ /dev/null @@ -1 +0,0 @@ -SELECT variance(*) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalidate_view1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalidate_view1.q deleted file mode 100644 index dd39c5eb4a4fe..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalidate_view1.q +++ /dev/null @@ -1,11 +0,0 @@ -DROP VIEW xxx8; -DROP VIEW xxx9; - --- create two levels of view reference, then invalidate intermediate view --- by dropping a column from underlying table, and verify that --- querying outermost view results in full error context -CREATE TABLE xxx10 (key int, value int); -CREATE VIEW xxx9 AS SELECT * FROM xxx10; -CREATE VIEW xxx8 AS SELECT * FROM xxx9 xxx; -ALTER TABLE xxx10 REPLACE COLUMNS (key int); -SELECT * FROM xxx8 yyy; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join2.q deleted file mode 100644 index 98a5f1e6629c7..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join2.q +++ /dev/null @@ -1,5 +0,0 @@ -SELECT /*+ MAPJOIN(x) */ x.key, x.value, y.value -FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key); - - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join28.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join28.q deleted file mode 100644 index 32ff105c2e450..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join28.q +++ /dev/null @@ -1,15 +0,0 @@ -CREATE TABLE dest_j1(key STRING, value STRING) STORED AS TEXTFILE; - --- Mapjoin followed by mapjoin is not supported. --- The same query would work fine without the hint. --- Note that there is a positive test with the same name in clientpositive -EXPLAIN -INSERT OVERWRITE TABLE dest_j1 -SELECT /*+ MAPJOIN(z) */ subq.key1, z.value -FROM -(SELECT /*+ MAPJOIN(x) */ x.key as key1, x.value as value1, y.key as key2, y.value as value2 - FROM src1 x JOIN src y ON (x.key = y.key)) subq - JOIN srcpart z ON (subq.key1 = z.key and z.ds='2008-04-08' and z.hr=11); - - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join29.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join29.q deleted file mode 100644 index 53a1652d25b20..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join29.q +++ /dev/null @@ -1,10 +0,0 @@ -CREATE TABLE dest_j1(key STRING, cnt1 INT, cnt2 INT); - --- Mapjoin followed by group by is not supported. --- The same query would work without the hint --- Note that there is a positive test with the same name in clientpositive -EXPLAIN -INSERT OVERWRITE TABLE dest_j1 -SELECT /*+ MAPJOIN(subq1) */ subq1.key, subq1.cnt, subq2.cnt -FROM (select x.key, count(1) as cnt from src1 x group by x.key) subq1 JOIN - (select y.key, count(1) as cnt from src y group by y.key) subq2 ON (subq1.key = subq2.key); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join32.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join32.q deleted file mode 100644 index 54a4dcd9afe2a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join32.q +++ /dev/null @@ -1,14 +0,0 @@ -CREATE TABLE dest_j1(key STRING, value STRING, val2 STRING) STORED AS TEXTFILE; - --- Mapjoin followed by Mapjoin is not supported. --- The same query would work without the hint --- Note that there is a positive test with the same name in clientpositive -EXPLAIN EXTENDED -INSERT OVERWRITE TABLE dest_j1 -SELECT /*+ MAPJOIN(x,z) */ x.key, z.value, y.value -FROM src1 x JOIN src y ON (x.key = y.key) -JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11); - - - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join35.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join35.q deleted file mode 100644 index fc8f77ca1232c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join35.q +++ /dev/null @@ -1,18 +0,0 @@ -CREATE TABLE dest_j1(key STRING, value STRING, val2 INT) STORED AS TEXTFILE; - --- Mapjoin followed by union is not supported. --- The same query would work without the hint --- Note that there is a positive test with the same name in clientpositive -EXPLAIN EXTENDED -INSERT OVERWRITE TABLE dest_j1 -SELECT /*+ MAPJOIN(x) */ x.key, x.value, subq1.cnt -FROM -( SELECT x.key as key, count(1) as cnt from src x where x.key < 20 group by x.key - UNION ALL - SELECT x1.key as key, count(1) as cnt from src x1 where x1.key > 100 group by x1.key -) subq1 -JOIN src1 x ON (x.key = subq1.key); - - - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_alt_syntax_comma_on.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_alt_syntax_comma_on.q deleted file mode 100644 index e39a38e2fcd47..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_alt_syntax_comma_on.q +++ /dev/null @@ -1,3 +0,0 @@ -explain select * -from src s1 , -src s2 on s1.key = s2.key; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous.q deleted file mode 100644 index c0da913c28812..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous.q +++ /dev/null @@ -1,6 +0,0 @@ - - -explain select s1.key, s2.key -from src s1, src s2 -where key = s2.key -; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous_vc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous_vc.q deleted file mode 100644 index 8e219637eb0c6..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous_vc.q +++ /dev/null @@ -1,5 +0,0 @@ - -explain select s1.key, s2.key -from src s1, src s2 -where INPUT__FILE__NAME = s2.INPUT__FILE__NAME -; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_nonexistent_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_nonexistent_part.q deleted file mode 100644 index b4a4757d22147..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_nonexistent_part.q +++ /dev/null @@ -1,4 +0,0 @@ -SET hive.security.authorization.enabled = true; -SELECT * -FROM srcpart s1 join src s2 on s1.key == s2.key -WHERE s1.ds='non-existent'; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/joinneg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/joinneg.q deleted file mode 100644 index a4967fd5dfb4a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/joinneg.q +++ /dev/null @@ -1,6 +0,0 @@ -EXPLAIN FROM -(SELECT src.* FROM src) x -JOIN -(SELECT src.* FROM src) Y -ON (x.key = b.key) -SELECT Y.*; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lateral_view_alias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lateral_view_alias.q deleted file mode 100644 index 50d535e6e1ecf..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lateral_view_alias.q +++ /dev/null @@ -1,3 +0,0 @@ --- Check alias count for LATERAL VIEW syntax: --- explode returns a table with only 1 col - should be an error if query specifies >1 col aliases -SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol1, myCol2 LIMIT 3; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lateral_view_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lateral_view_join.q deleted file mode 100644 index 818754ecbf05b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lateral_view_join.q +++ /dev/null @@ -1 +0,0 @@ -SELECT src.key FROM src LATERAL VIEW explode(array(1,2,3)) AS myTable JOIN src b ON src.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition.q deleted file mode 100644 index d59394544ccf0..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition.q +++ /dev/null @@ -1,7 +0,0 @@ -set hive.limit.query.max.table.partition=1; - -explain select * from srcpart limit 1; -select * from srcpart limit 1; - -explain select * from srcpart; -select * from srcpart; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition_stats.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition_stats.q deleted file mode 100644 index 0afd4a965ab94..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition_stats.q +++ /dev/null @@ -1,18 +0,0 @@ -set hive.exec.dynamic.partition=true; -set hive.exec.dynamic.partition.mode=nonstrict; -set hive.stats.autogather=true; -set hive.compute.query.using.stats=true; - -create table part (c int) partitioned by (d string); -insert into table part partition (d) -select hr,ds from srcpart; - -set hive.limit.query.max.table.partition=1; - -explain select count(*) from part; -select count(*) from part; - -set hive.compute.query.using.stats=false; - -explain select count(*) from part; -select count(*) from part; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/line_terminator.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/line_terminator.q deleted file mode 100644 index ad3542c40ace4..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/line_terminator.q +++ /dev/null @@ -1,3 +0,0 @@ -CREATE TABLE mytable (col1 STRING, col2 INT) -ROW FORMAT DELIMITED -LINES TERMINATED BY ','; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_exist_part_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_exist_part_authfail.q deleted file mode 100644 index eb72d940a5392..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_exist_part_authfail.q +++ /dev/null @@ -1,4 +0,0 @@ -create table hive_test_src ( col1 string ) partitioned by (pcol1 string) stored as textfile; -alter table hive_test_src add partition (pcol1 = 'test_part'); -set hive.security.authorization.enabled=true; -load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src partition (pcol1 = 'test_part'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_non_native.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_non_native.q deleted file mode 100644 index 75a5216e00d82..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_non_native.q +++ /dev/null @@ -1,5 +0,0 @@ - -CREATE TABLE non_native2(key int, value string) -STORED BY 'org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler'; - -LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE non_native2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_nonpart_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_nonpart_authfail.q deleted file mode 100644 index 32653631ad6a4..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_nonpart_authfail.q +++ /dev/null @@ -1,3 +0,0 @@ -create table hive_test_src ( col1 string ) stored as textfile; -set hive.security.authorization.enabled=true; -load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src ; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_authfail.q deleted file mode 100644 index 315988dc0a959..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_authfail.q +++ /dev/null @@ -1,3 +0,0 @@ -create table hive_test_src ( col1 string ) partitioned by (pcol1 string) stored as textfile; -set hive.security.authorization.enabled=true; -load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src partition (pcol1 = 'test_part'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_nospec.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_nospec.q deleted file mode 100644 index 81517991b26fa..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_nospec.q +++ /dev/null @@ -1,2 +0,0 @@ -create table hive_test_src ( col1 string ) partitioned by (pcol1 string) stored as textfile; -load data local inpath '../../data/files/test.dat' into table hive_test_src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_stored_as_dirs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_stored_as_dirs.q deleted file mode 100644 index c56f0d408d4ad..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_stored_as_dirs.q +++ /dev/null @@ -1,7 +0,0 @@ -set hive.mapred.supports.subdirectories=true; - --- Load data can't work with table with stored as directories -CREATE TABLE if not exists stored_as_dirs_multiple (col1 STRING, col2 int, col3 STRING) -SKEWED BY (col1, col2) ON (('s1',1), ('s3',3), ('s13',13), ('s78',78)) stored as DIRECTORIES; - -LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE stored_as_dirs_multiple; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_view_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_view_failure.q deleted file mode 100644 index 64182eac8362d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_view_failure.q +++ /dev/null @@ -1,3 +0,0 @@ -DROP VIEW xxx11; -CREATE VIEW xxx11 AS SELECT * FROM src; -LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE xxx11; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat.q deleted file mode 100644 index f0c3b59d30ddf..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat.q +++ /dev/null @@ -1,6 +0,0 @@ --- test for loading into tables with the correct file format --- test for loading into partitions with the correct file format - - -CREATE TABLE load_wrong_fileformat_T1(name STRING) STORED AS SEQUENCEFILE; -LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE load_wrong_fileformat_T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q deleted file mode 100644 index 4d79bbeb102c7..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q +++ /dev/null @@ -1,6 +0,0 @@ --- test for loading into tables with the correct file format --- test for loading into partitions with the correct file format - - -CREATE TABLE T1(name STRING) STORED AS RCFILE; -LOAD DATA LOCAL INPATH '../../data/files/kv1.seq' INTO TABLE T1; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q deleted file mode 100644 index 050c819a2f04b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q +++ /dev/null @@ -1,6 +0,0 @@ --- test for loading into tables with the correct file format --- test for loading into partitions with the correct file format - - -CREATE TABLE T1(name STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../../data/files/kv1.seq' INTO TABLE T1; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_noof_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_noof_part.q deleted file mode 100644 index 7f5ad754142ab..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_noof_part.q +++ /dev/null @@ -1,3 +0,0 @@ - -CREATE TABLE loadpart1(a STRING, b STRING) PARTITIONED BY (ds STRING,ds1 STRING); -LOAD DATA LOCAL INPATH '../../data1/files/kv1.txt' INTO TABLE loadpart1 PARTITION(ds='2009-05-05'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/local_mapred_error_cache.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/local_mapred_error_cache.q deleted file mode 100644 index ed9e21dd8a1fb..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/local_mapred_error_cache.q +++ /dev/null @@ -1,4 +0,0 @@ -set hive.exec.mode.local.auto=true; -set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifySessionStateLocalErrorsHook; - -FROM src SELECT TRANSFORM(key, value) USING 'python ../../data/scripts/cat_error.py' AS (key, value); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg1.q deleted file mode 100644 index e1b58fca80af3..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg1.q +++ /dev/null @@ -1,10 +0,0 @@ -drop table tstsrc; -create table tstsrc like src; -insert overwrite table tstsrc select key, value from src; - -set hive.lock.numretries=0; -set hive.unlock.numretries=0; - -LOCK TABLE tstsrc SHARED; -LOCK TABLE tstsrc SHARED; -LOCK TABLE tstsrc EXCLUSIVE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg2.q deleted file mode 100644 index a4604cd470658..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg2.q +++ /dev/null @@ -1,6 +0,0 @@ -drop table tstsrc; -create table tstsrc like src; -insert overwrite table tstsrc select key, value from src; - -set hive.unlock.numretries=0; -UNLOCK TABLE tstsrc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg3.q deleted file mode 100644 index f2252f7bdf4da..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg3.q +++ /dev/null @@ -1,9 +0,0 @@ -drop table tstsrcpart; -create table tstsrcpart like srcpart; - -insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='11') -select key, value from srcpart where ds='2008-04-08' and hr='11'; - -set hive.lock.numretries=0; -set hive.unlock.numretries=0; -UNLOCK TABLE tstsrcpart PARTITION(ds='2008-04-08', hr='11'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg4.q deleted file mode 100644 index b47644cca362c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg4.q +++ /dev/null @@ -1,12 +0,0 @@ -drop table tstsrcpart; -create table tstsrcpart like srcpart; - -insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='11') -select key, value from srcpart where ds='2008-04-08' and hr='11'; - -set hive.lock.numretries=0; -set hive.unlock.numretries=0; - -LOCK TABLE tstsrcpart PARTITION(ds='2008-04-08', hr='11') EXCLUSIVE; -SHOW LOCKS tstsrcpart PARTITION(ds='2008-04-08', hr='12'); - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg5.q deleted file mode 100644 index 19c1ce28c2422..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg5.q +++ /dev/null @@ -1,2 +0,0 @@ -drop table tstsrcpart; -show locks tstsrcpart extended; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_query_tbl_in_locked_db.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_query_tbl_in_locked_db.q deleted file mode 100644 index 4966f2b9b2825..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_query_tbl_in_locked_db.q +++ /dev/null @@ -1,17 +0,0 @@ -create database lockneg1; -use lockneg1; - -create table tstsrcpart like default.srcpart; - -insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='11') -select key, value from default.srcpart where ds='2008-04-08' and hr='11'; - -lock database lockneg1 shared; -show locks database lockneg1; -select count(1) from tstsrcpart where ds='2008-04-08' and hr='11'; - -unlock database lockneg1; -show locks database lockneg1; -lock database lockneg1 exclusive; -show locks database lockneg1; -select count(1) from tstsrcpart where ds='2008-04-08' and hr='11'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_db_lock_conflict.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_db_lock_conflict.q deleted file mode 100644 index 1f9ad90898dce..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_db_lock_conflict.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.lock.numretries=0; - -create database lockneg4; - -lock database lockneg4 exclusive; -lock database lockneg4 shared; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_drop_locked_db.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_drop_locked_db.q deleted file mode 100644 index 8cbe31083b400..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_drop_locked_db.q +++ /dev/null @@ -1,8 +0,0 @@ -set hive.lock.numretries=0; - -create database lockneg9; - -lock database lockneg9 shared; -show locks database lockneg9; - -drop database lockneg9; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_lock_db_in_use.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_lock_db_in_use.q deleted file mode 100644 index 4127a6f150a13..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_lock_db_in_use.q +++ /dev/null @@ -1,15 +0,0 @@ -set hive.lock.numretries=0; - -create database lockneg2; -use lockneg2; - -create table tstsrcpart like default.srcpart; - -insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='11') -select key, value from default.srcpart where ds='2008-04-08' and hr='11'; - -lock table tstsrcpart shared; -show locks; - -lock database lockneg2 exclusive; -show locks; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/macro_unused_parameter.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/macro_unused_parameter.q deleted file mode 100644 index 523710ddf3a5b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/macro_unused_parameter.q +++ /dev/null @@ -1 +0,0 @@ -CREATE TEMPORARY MACRO BAD_MACRO (x INT, y INT) x; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace.q deleted file mode 100644 index 76c7ae94d4b63..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.exec.mode.local.auto=false; -set hive.exec.job.debug.capture.stacktraces=true; -set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifySessionStateStackTracesHook; - -FROM src SELECT TRANSFORM(key, value) USING 'script_does_not_exist' AS (key, value); - --- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) --- Hadoop 0.23 changes the getTaskDiagnostics behavior --- The Error Code of hive failure MapReduce job changes --- In Hadoop 0.20 --- Hive failure MapReduce job gets 20000 as Error Code --- In Hadoop 0.23 --- Hive failure MapReduce job gets 2 as Error Code diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_hadoop20.q deleted file mode 100644 index 9d0548cc10f55..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_hadoop20.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.exec.mode.local.auto=false; -set hive.exec.job.debug.capture.stacktraces=true; -set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifySessionStateStackTracesHook; - -FROM src SELECT TRANSFORM(key, value) USING 'script_does_not_exist' AS (key, value); - --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.23) --- Hadoop 0.23 changes the getTaskDiagnostics behavior --- The Error Code of hive failure MapReduce job changes --- In Hadoop 0.20 --- Hive failure MapReduce job gets 20000 as Error Code --- In Hadoop 0.23 --- Hive failure MapReduce job gets 2 as Error Code diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff.q deleted file mode 100644 index c93aedb3137bf..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.exec.mode.local.auto=false; -set hive.exec.job.debug.capture.stacktraces=false; -set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifySessionStateStackTracesHook; - -FROM src SELECT TRANSFORM(key, value) USING 'script_does_not_exist' AS (key, value); - --- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) --- Hadoop 0.23 changes the getTaskDiagnostics behavior --- The Error Code of hive failure MapReduce job changes --- In Hadoop 0.20 --- Hive failure MapReduce job gets 20000 as Error Code --- In Hadoop 0.23 --- Hive failure MapReduce job gets 2 as Error Code diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q deleted file mode 100644 index e319944958c2a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q +++ /dev/null @@ -1,13 +0,0 @@ -set hive.exec.mode.local.auto=false; -set hive.exec.job.debug.capture.stacktraces=false; -set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifySessionStateStackTracesHook; - -FROM src SELECT TRANSFORM(key, value) USING 'script_does_not_exist' AS (key, value); - --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.23) --- Hadoop 0.23 changes the getTaskDiagnostics behavior --- The Error Code of hive failure MapReduce job changes --- In Hadoop 0.20 --- Hive failure MapReduce job gets 20000 as Error Code --- In Hadoop 0.23 --- Hive failure MapReduce job gets 2 as Error Code diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_1.q deleted file mode 100644 index 0a48c01546ec5..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_1.q +++ /dev/null @@ -1,3 +0,0 @@ -create table src2 like src; -CREATE INDEX src_index_merge_test ON TABLE src2(key) as 'COMPACT' WITH DEFERRED REBUILD; -alter table src2 concatenate; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_2.q deleted file mode 100644 index a4fab1c8b804d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_2.q +++ /dev/null @@ -1,3 +0,0 @@ -create table srcpart2 (key int, value string) partitioned by (ds string); -insert overwrite table srcpart2 partition (ds='2011') select * from src; -alter table srcpart2 concatenate; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_3.q deleted file mode 100644 index 6bc645e4c2378..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_3.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.enforce.bucketing=true; -set hive.enforce.sorting=true; - -create table srcpart2 (key int, value string) partitioned by (ds string) clustered by (key) sorted by (key) into 2 buckets stored as RCFILE; -insert overwrite table srcpart2 partition (ds='2011') select * from src; -alter table srcpart2 partition (ds = '2011') concatenate; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/minimr_broken_pipe.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/minimr_broken_pipe.q deleted file mode 100644 index 8dda9cdf4a37d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/minimr_broken_pipe.q +++ /dev/null @@ -1,4 +0,0 @@ -set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; -set hive.exec.script.allow.partial.consumption = false; --- Tests exception in ScriptOperator.close() by passing to the operator a small amount of data -SELECT TRANSFORM(*) USING 'true' AS a, b FROM (SELECT TRANSFORM(*) USING 'echo' AS a, b FROM src LIMIT 1) tmp; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nested_complex_neg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nested_complex_neg.q deleted file mode 100644 index 09f13f52aeadf..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nested_complex_neg.q +++ /dev/null @@ -1,15 +0,0 @@ - -create table nestedcomplex ( -simple_int int, -max_nested_array array>>>>>>>>>>>>>>>>>>>>>>, -max_nested_map array>>>>>>>>>>>>>>>>>>>>>, -max_nested_struct array>>>>>>>>>>>>>>>>>>>>>>, -simple_string string) - -; - - --- This should fail in as extended nesting levels are not enabled using the serdeproperty hive.serialization.extend.nesting.levels -load data local inpath '../../data/files/nested_complex.txt' overwrite into table nestedcomplex; - -select * from nestedcomplex sort by simple_int; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/no_matching_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/no_matching_udf.q deleted file mode 100644 index 0c24b1626a532..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/no_matching_udf.q +++ /dev/null @@ -1 +0,0 @@ -SELECT percentile(3.5, 0.99) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nonkey_groupby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nonkey_groupby.q deleted file mode 100644 index 431e04efd9342..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nonkey_groupby.q +++ /dev/null @@ -1 +0,0 @@ -EXPLAIN SELECT key, count(1) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_insert.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_insert.q deleted file mode 100644 index 6669bf62d8822..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_insert.q +++ /dev/null @@ -1,7 +0,0 @@ - -CREATE TABLE nopart_insert(a STRING, b STRING) PARTITIONED BY (ds STRING); - -INSERT OVERWRITE TABLE nopart_insert -SELECT TRANSFORM(src.key, src.value) USING '../../data/scripts/error_script' AS (tkey, tvalue) -FROM src; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_load.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_load.q deleted file mode 100644 index 966982fd5ce52..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_load.q +++ /dev/null @@ -1,5 +0,0 @@ - -CREATE TABLE nopart_load(a STRING, b STRING) PARTITIONED BY (ds STRING); - -load data local inpath '../../data/files/kv1.txt' overwrite into table nopart_load ; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/notable_alias4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/notable_alias4.q deleted file mode 100644 index e7ad6b79d3ed9..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/notable_alias4.q +++ /dev/null @@ -1,4 +0,0 @@ -EXPLAIN -SELECT key from src JOIN src1 on src1.key=src.key; - -SELECT key from src JOIN src1 on src1.key=src.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderby_invalid_position.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderby_invalid_position.q deleted file mode 100644 index 4dbf2a6d56a2e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderby_invalid_position.q +++ /dev/null @@ -1,4 +0,0 @@ -set hive.groupby.orderby.position.alias=true; - --- invalid position alias in order by -SELECT src.key, src.value FROM src ORDER BY 0; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderby_position_unsupported.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderby_position_unsupported.q deleted file mode 100644 index a490c2306ec4a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderby_position_unsupported.q +++ /dev/null @@ -1,4 +0,0 @@ -set hive.groupby.orderby.position.alias=true; - --- position alias is not supported when SELECT * -SELECT src.* FROM src ORDER BY 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderbysortby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderbysortby.q deleted file mode 100644 index 5dff69fdbb78d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderbysortby.q +++ /dev/null @@ -1,8 +0,0 @@ -CREATE TABLE dest1(key INT, ten INT, one INT, value STRING) STORED AS TEXTFILE; - -FROM src -INSERT OVERWRITE TABLE dest1 -MAP src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value -USING 'cat' AS (tkey, ten, one, tvalue) -ORDER BY tvalue, tkey -SORT BY ten, one; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_char.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_char.q deleted file mode 100644 index 745a7867264e3..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_char.q +++ /dev/null @@ -1,3 +0,0 @@ -drop table if exists parquet_char; - -create table parquet_char (t char(10)) stored as parquet; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_date.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_date.q deleted file mode 100644 index 89d3602fd3e97..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_date.q +++ /dev/null @@ -1,3 +0,0 @@ -drop table if exists parquet_date; - -create table parquet_date (t date) stored as parquet; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_decimal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_decimal.q deleted file mode 100644 index 8a4973110a51f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_decimal.q +++ /dev/null @@ -1,3 +0,0 @@ -drop table if exists parquet_decimal; - -create table parquet_decimal (t decimal(4,2)) stored as parquet; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_timestamp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_timestamp.q deleted file mode 100644 index 4ef36fa0efc49..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_timestamp.q +++ /dev/null @@ -1,3 +0,0 @@ -drop table if exists parquet_timestamp; - -create table parquet_timestamp (t timestamp) stored as parquet; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_varchar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_varchar.q deleted file mode 100644 index 55825f76dc240..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_varchar.q +++ /dev/null @@ -1,3 +0,0 @@ -drop table if exists parquet_varchar; - -create table parquet_varchar (t varchar(10)) stored as parquet; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/part_col_complex_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/part_col_complex_type.q deleted file mode 100644 index 4b9eb847db542..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/part_col_complex_type.q +++ /dev/null @@ -1 +0,0 @@ -create table t (a string) partitioned by (b map); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part.q deleted file mode 100644 index 541599915afc0..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part.q +++ /dev/null @@ -1,15 +0,0 @@ --- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode - -drop table tbl_protectmode3; - -create table tbl_protectmode3 (col string) partitioned by (p string); -alter table tbl_protectmode3 add partition (p='p1'); -alter table tbl_protectmode3 add partition (p='p2'); - -select * from tbl_protectmode3 where p='p1'; -select * from tbl_protectmode3 where p='p2'; - -alter table tbl_protectmode3 partition (p='p1') enable offline; - -select * from tbl_protectmode3 where p='p2'; -select * from tbl_protectmode3 where p='p1'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part1.q deleted file mode 100644 index 99256da285c1a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part1.q +++ /dev/null @@ -1,21 +0,0 @@ --- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode - -drop table tbl_protectmode5; - -create table tbl_protectmode5_1 (col string); - -create table tbl_protectmode5 (col string) partitioned by (p string); -alter table tbl_protectmode5 add partition (p='p1'); -alter table tbl_protectmode5 add partition (p='p2'); - -insert overwrite table tbl_protectmode5_1 -select col from tbl_protectmode5 where p='p1'; -insert overwrite table tbl_protectmode5_1 -select col from tbl_protectmode5 where p='p2'; - -alter table tbl_protectmode5 partition (p='p1') enable offline; - -insert overwrite table tbl_protectmode5_1 -select col from tbl_protectmode5 where p='p2'; -insert overwrite table tbl_protectmode5_1 -select col from tbl_protectmode5 where p='p1'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part2.q deleted file mode 100644 index 3fdc036996563..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part2.q +++ /dev/null @@ -1,9 +0,0 @@ --- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode - -drop table tbl_protectmode6; - -create table tbl_protectmode6 (c1 string,c2 string) partitioned by (p string); -alter table tbl_protectmode6 add partition (p='p1'); -LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' OVERWRITE INTO TABLE tbl_protectmode6 partition (p='p1'); -alter table tbl_protectmode6 partition (p='p1') enable offline; -LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' OVERWRITE INTO TABLE tbl_protectmode6 partition (p='p1'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q deleted file mode 100644 index b4e508ff98180..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q +++ /dev/null @@ -1,10 +0,0 @@ --- protect mode: syntax to change protect mode works and queries to drop partitions are blocked if it is marked no drop - -drop table tbl_protectmode_no_drop; - -create table tbl_protectmode_no_drop (c1 string,c2 string) partitioned by (p string); -alter table tbl_protectmode_no_drop add partition (p='p1'); -alter table tbl_protectmode_no_drop partition (p='p1') enable no_drop; -desc extended tbl_protectmode_no_drop partition (p='p1'); - -alter table tbl_protectmode_no_drop drop partition (p='p1'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl1.q deleted file mode 100644 index 236129902c07c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl1.q +++ /dev/null @@ -1,8 +0,0 @@ --- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode - -drop table tbl_protectmode_1; - -create table tbl_protectmode_1 (col string); -select * from tbl_protectmode_1; -alter table tbl_protectmode_1 enable offline; -select * from tbl_protectmode_1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl2.q deleted file mode 100644 index 05964c35e9e08..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl2.q +++ /dev/null @@ -1,12 +0,0 @@ --- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode - -drop table tbl_protectmode2; - -create table tbl_protectmode2 (col string) partitioned by (p string); -alter table tbl_protectmode2 add partition (p='p1'); -alter table tbl_protectmode2 enable no_drop; -alter table tbl_protectmode2 enable offline; -alter table tbl_protectmode2 disable no_drop; -desc extended tbl_protectmode2; - -select * from tbl_protectmode2 where p='p1'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl3.q deleted file mode 100644 index bbaa2670875b6..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl3.q +++ /dev/null @@ -1,10 +0,0 @@ --- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode - -drop table tbl_protectmode_4; - -create table tbl_protectmode_4 (col string); -select col from tbl_protectmode_4; -alter table tbl_protectmode_4 enable offline; -desc extended tbl_protectmode_4; - -select col from tbl_protectmode_4; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl4.q deleted file mode 100644 index c7880de6d8aec..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl4.q +++ /dev/null @@ -1,15 +0,0 @@ --- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode - -drop table tbl_protectmode_tbl4; -drop table tbl_protectmode_tbl4_src; - -create table tbl_protectmode_tbl4_src (col string); - -create table tbl_protectmode_tbl4 (col string) partitioned by (p string); -alter table tbl_protectmode_tbl4 add partition (p='p1'); -alter table tbl_protectmode_tbl4 enable no_drop; -alter table tbl_protectmode_tbl4 enable offline; -alter table tbl_protectmode_tbl4 disable no_drop; -desc extended tbl_protectmode_tbl4; - -select col from tbl_protectmode_tbl4 where p='not_exist'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl5.q deleted file mode 100644 index cd848fd4a1b9f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl5.q +++ /dev/null @@ -1,15 +0,0 @@ --- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode - -drop table tbl_protectmode_tbl5; -drop table tbl_protectmode_tbl5_src; - -create table tbl_protectmode_tbl5_src (col string); - -create table tbl_protectmode_tbl5 (col string) partitioned by (p string); -alter table tbl_protectmode_tbl5 add partition (p='p1'); -alter table tbl_protectmode_tbl5 enable no_drop; -alter table tbl_protectmode_tbl5 enable offline; -alter table tbl_protectmode_tbl5 disable no_drop; -desc extended tbl_protectmode_tbl5; - -insert overwrite table tbl_protectmode_tbl5 partition (p='not_exist') select col from tbl_protectmode_tbl5_src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl6.q deleted file mode 100644 index 26248cc6b4877..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl6.q +++ /dev/null @@ -1,8 +0,0 @@ --- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode - -drop table tbl_protectmode_tbl6; - -create table tbl_protectmode_tbl6 (col string); -alter table tbl_protectmode_tbl6 enable no_drop cascade; - -drop table tbl_protectmode_tbl6; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl7.q deleted file mode 100644 index afff8404edc01..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl7.q +++ /dev/null @@ -1,13 +0,0 @@ --- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode - -drop table tbl_protectmode_tbl7; -create table tbl_protectmode_tbl7 (col string) partitioned by (p string); -alter table tbl_protectmode_tbl7 add partition (p='p1'); -alter table tbl_protectmode_tbl7 enable no_drop; - -alter table tbl_protectmode_tbl7 drop partition (p='p1'); - -alter table tbl_protectmode_tbl7 add partition (p='p1'); -alter table tbl_protectmode_tbl7 enable no_drop cascade; - -alter table tbl_protectmode_tbl7 drop partition (p='p1'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl8.q deleted file mode 100644 index 809c287fc502a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl8.q +++ /dev/null @@ -1,13 +0,0 @@ --- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode - -drop table tbl_protectmode_tbl8; -create table tbl_protectmode_tbl8 (col string) partitioned by (p string); -alter table tbl_protectmode_tbl8 add partition (p='p1'); -alter table tbl_protectmode_tbl8 enable no_drop; - -alter table tbl_protectmode_tbl8 drop partition (p='p1'); - -alter table tbl_protectmode_tbl8 enable no_drop cascade; - -alter table tbl_protectmode_tbl8 add partition (p='p1'); -alter table tbl_protectmode_tbl8 drop partition (p='p1'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q deleted file mode 100644 index a4ef2acbfd406..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q +++ /dev/null @@ -1,9 +0,0 @@ --- protect mode: syntax to change protect mode works and queries are not blocked if a table or partition is not in protect mode - -drop table tbl_protectmode__no_drop; - -create table tbl_protectmode__no_drop (col string); -select * from tbl_protectmode__no_drop; -alter table tbl_protectmode__no_drop enable no_drop; -desc extended tbl_protectmode__no_drop; -drop table tbl_protectmode__no_drop; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q deleted file mode 100644 index ef372259ed3e3..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q +++ /dev/null @@ -1,20 +0,0 @@ -DROP TABLE part; - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - -LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; - --- testAggrFuncsWithNoGBYNoPartDef -select p_mfgr, -sum(p_retailprice) as s1 -from part; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q deleted file mode 100644 index 58430423436b7..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q +++ /dev/null @@ -1,28 +0,0 @@ -DROP TABLE part; - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - -LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; - --- testAmbiguousWindowDefn -select p_mfgr, p_name, p_size, -sum(p_size) over (w1) as s1, -sum(p_size) over (w2) as s2, -sum(p_size) over (w3) as s3 -from part -distribute by p_mfgr -sort by p_mfgr -window w1 as (rows between 2 preceding and 2 following), - w2 as (rows between unbounded preceding and current row), - w3 as w3; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_DistributeByOrderBy.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_DistributeByOrderBy.q deleted file mode 100644 index caebebf8eaa4c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_DistributeByOrderBy.q +++ /dev/null @@ -1,19 +0,0 @@ -DROP TABLE part; - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - --- testPartitonBySortBy -select p_mfgr, p_name, p_size, -sum(p_retailprice) over (distribute by p_mfgr order by p_mfgr) as s1 -from part -; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_DuplicateWindowAlias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_DuplicateWindowAlias.q deleted file mode 100644 index 3a0304188d2ac..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_DuplicateWindowAlias.q +++ /dev/null @@ -1,22 +0,0 @@ -DROP TABLE part; - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - --- testDuplicateWindowAlias -select p_mfgr, p_name, p_size, -sum(p_size) over (w1) as s1, -sum(p_size) over (w2) as s2 -from part -window w1 as (partition by p_mfgr order by p_mfgr rows between 2 preceding and 2 following), - w2 as w1, - w2 as (rows between unbounded preceding and current row); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q deleted file mode 100644 index f351a1448b150..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q +++ /dev/null @@ -1,20 +0,0 @@ -DROP TABLE part; - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - --- testHavingLeadWithNoGBYNoWindowing -select p_mfgr,p_name, p_size -from part -having lead(p_size, 1) over() <= p_size -distribute by p_mfgr -sort by p_name; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithPTF.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithPTF.q deleted file mode 100644 index d0d3d3fae23f4..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithPTF.q +++ /dev/null @@ -1,22 +0,0 @@ -DROP TABLE part; - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - --- testHavingLeadWithPTF -select p_mfgr,p_name, p_size -from noop(on part -partition by p_mfgr -order by p_name) -having lead(p_size, 1) over() <= p_size -distribute by p_mfgr -sort by p_name; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_InvalidValueBoundary.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_InvalidValueBoundary.q deleted file mode 100644 index 40a39cb68b5e3..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_InvalidValueBoundary.q +++ /dev/null @@ -1,21 +0,0 @@ -DROP TABLE part; - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING, - p_complex array -); - --- testInvalidValueBoundary -select p_mfgr,p_name, p_size, -sum(p_size) over (w1) as s , -dense_rank() over(w1) as dr -from part -window w1 as (partition by p_mfgr order by p_complex range between 2 preceding and current row); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_JoinWithAmbigousAlias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_JoinWithAmbigousAlias.q deleted file mode 100644 index 80441e4f571f8..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_JoinWithAmbigousAlias.q +++ /dev/null @@ -1,20 +0,0 @@ -DROP TABLE part; - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - --- testJoinWithAmbigousAlias -select abc.* -from noop(on part -partition by p_mfgr -order by p_name -) abc join part on abc.p_partkey = p1.p_partkey; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_PartitionBySortBy.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_PartitionBySortBy.q deleted file mode 100644 index 1c98b8743cd72..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_PartitionBySortBy.q +++ /dev/null @@ -1,19 +0,0 @@ -DROP TABLE part; - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - --- testPartitonBySortBy -select p_mfgr, p_name, p_size, -sum(p_retailprice) over (partition by p_mfgr sort by p_mfgr) as s1 -from part -; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_WhereWithRankCond.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_WhereWithRankCond.q deleted file mode 100644 index 8f4a21bd6c966..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_WhereWithRankCond.q +++ /dev/null @@ -1,21 +0,0 @@ -DROP TABLE part; - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - --- testWhereWithRankCond -select p_mfgr,p_name, p_size, -rank() over() as r -from part -where r < 4 -distribute by p_mfgr -sort by p_mfgr; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_window_boundaries.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_window_boundaries.q deleted file mode 100644 index ddab4367bb66b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_window_boundaries.q +++ /dev/null @@ -1,17 +0,0 @@ --- data setup -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - -select p_mfgr, p_name, p_size, - sum(p_retailprice) over (rows unbounded following) as s1 - from part distribute by p_mfgr sort by p_name; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_window_boundaries2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_window_boundaries2.q deleted file mode 100644 index 16cb52ca8414e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_window_boundaries2.q +++ /dev/null @@ -1,17 +0,0 @@ --- data setup -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - -select p_mfgr, p_name, p_size, - sum(p_retailprice) over (range unbounded following) as s1 - from part distribute by p_mfgr sort by p_name; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/recursive_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/recursive_view.q deleted file mode 100644 index 590523e9b6259..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/recursive_view.q +++ /dev/null @@ -1,15 +0,0 @@ --- Can't have recursive views - -drop table t; -drop view r0; -drop view r1; -drop view r2; -drop view r3; -create table t (id int); -create view r0 as select * from t; -create view r1 as select * from r0; -create view r2 as select * from r1; -create view r3 as select * from r2; -drop view r0; -alter view r3 rename to r0; -select * from r0; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_1.q deleted file mode 100644 index a171961a683ee..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_1.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.support.quoted.identifiers=none; -EXPLAIN -SELECT `+++` FROM srcpart; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_2.q deleted file mode 100644 index 7bac1c775522b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_2.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.support.quoted.identifiers=none; -EXPLAIN -SELECT `.a.` FROM srcpart; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_groupby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_groupby.q deleted file mode 100644 index 300d145508887..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_groupby.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.support.quoted.identifiers=none; -EXPLAIN -SELECT `..`, count(1) FROM srcpart GROUP BY `..`; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sa_fail_hook3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sa_fail_hook3.q deleted file mode 100644 index e54201c09e6f5..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sa_fail_hook3.q +++ /dev/null @@ -1,4 +0,0 @@ -create table mp2 (a string) partitioned by (b string); -alter table mp2 add partition (b='1'); -alter table mp2 partition (b='1') enable NO_DROP; -alter table mp2 drop partition (b='1'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sample.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sample.q deleted file mode 100644 index 0086352f8c47c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sample.q +++ /dev/null @@ -1 +0,0 @@ -explain extended SELECT s.* FROM srcbucket TABLESAMPLE (BUCKET 5 OUT OF 4 on key) s \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe2.q deleted file mode 100644 index 1c3093c0e7023..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe2.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.exec.script.allow.partial.consumption = false; --- Tests exception in ScriptOperator.processOp() by passing extra data needed to fill pipe buffer -SELECT TRANSFORM(key, value, key, value, key, value, key, value, key, value, key, value, key, value, key, value, key, value, key, value, key, value, key, value) USING 'true' as a,b,c,d FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe3.q deleted file mode 100644 index 60f93d209802a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe3.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.exec.script.allow.partial.consumption = true; --- Test to ensure that a script with a bad error code still fails even with partial consumption -SELECT TRANSFORM(*) USING 'false' AS a, b FROM (SELECT TRANSFORM(*) USING 'echo' AS a, b FROM src LIMIT 1) tmp; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_error.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_error.q deleted file mode 100644 index 8ca849b82d8ad..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_error.q +++ /dev/null @@ -1,7 +0,0 @@ -EXPLAIN -SELECT TRANSFORM(src.key, src.value) USING '../../data/scripts/error_script' AS (tkey, tvalue) -FROM src; - -SELECT TRANSFORM(src.key, src.value) USING '../../data/scripts/error_script' AS (tkey, tvalue) -FROM src; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/select_charliteral.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/select_charliteral.q deleted file mode 100644 index 1e4c70e663f05..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/select_charliteral.q +++ /dev/null @@ -1,3 +0,0 @@ --- Check that charSetLiteral syntax conformance --- Check that a sane error message with correct line/column numbers is emitted with helpful context tokens. -select _c17, count(1) from tmp_tl_foo group by _c17 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/select_udtf_alias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/select_udtf_alias.q deleted file mode 100644 index 8ace4414fc14e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/select_udtf_alias.q +++ /dev/null @@ -1,3 +0,0 @@ --- Check alias count for SELECT UDTF() syntax: --- explode returns a table with only 1 col - should be an error if query specifies >1 col aliases -SELECT explode(array(1,2,3)) AS (myCol1, myCol2) LIMIT 3; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin1.q deleted file mode 100644 index 06e6cad34b4d8..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin1.q +++ /dev/null @@ -1,2 +0,0 @@ --- reference rhs of semijoin in select-clause -select b.value from src a left semi join src b on (b.key = a.key and b.key = '100'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin2.q deleted file mode 100644 index 46faae641640d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin2.q +++ /dev/null @@ -1,2 +0,0 @@ --- rhs table reference in the where clause -select a.value from src a left semi join src b on a.key = b.key where b.value = 'val_18'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin3.q deleted file mode 100644 index 35b455a7292d1..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin3.q +++ /dev/null @@ -1,2 +0,0 @@ --- rhs table reference in group by -select * from src a left semi join src b on a.key = b.key group by b.value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin4.q deleted file mode 100644 index 4e52ebfb3cdeb..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin4.q +++ /dev/null @@ -1,3 +0,0 @@ --- rhs table is a view and reference the view in where clause -select a.value from src a left semi join (select key , value from src where key > 100) b on a.key = b.key where b.value = 'val_108' ; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex.q deleted file mode 100644 index 13b3f165b9684..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex.q +++ /dev/null @@ -1,17 +0,0 @@ -USE default; --- This should fail because Regex SerDe doesn't support STRUCT -CREATE TABLE serde_regex( - host STRING, - identity STRING, - user STRING, - time TIMESTAMP, - request STRING, - status INT, - size INT, - referer STRING, - agent STRING, - strct STRUCT) -ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.RegexSerDe' -WITH SERDEPROPERTIES ( - "input.regex" = "([^ ]*) ([^ ]*) ([^ ]*) (-|\\[[^\\]]*\\]) ([^ \"]*|\"[^\"]*\") (-|[0-9]*) (-|[0-9]*)(?: ([^ \"]*|\"[^\"]*\") ([^ \"]*|\"[^\"]*\"))?") -STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex2.q deleted file mode 100644 index d523d03e906c1..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex2.q +++ /dev/null @@ -1,23 +0,0 @@ -USE default; --- Mismatch between the number of matching groups and columns, throw run time exception. Ideally this should throw a compile time exception. See JIRA-3023 for more details. - CREATE TABLE serde_regex( - host STRING, - identity STRING, - user STRING, - time STRING, - request STRING, - status STRING, - size STRING, - referer STRING, - agent STRING) -ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.RegexSerDe' -WITH SERDEPROPERTIES ( - "input.regex" = "([^ ]*) ([^ ]*) ([^ ]*) (-|\\[[^\\]]*\\]) ([^ \"]*|\"[^\"]*\") (-|[0-9]*) (-|[0-9]*)" -) -STORED AS TEXTFILE; - -LOAD DATA LOCAL INPATH "../../data/files/apache.access.log" INTO TABLE serde_regex; -LOAD DATA LOCAL INPATH "../../data/files/apache.access.2.log" INTO TABLE serde_regex; - --- raise an exception -SELECT * FROM serde_regex ORDER BY time; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex3.q deleted file mode 100644 index 5a0295c971c26..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex3.q +++ /dev/null @@ -1,14 +0,0 @@ -USE default; --- null input.regex, raise an exception - CREATE TABLE serde_regex( - host STRING, - identity STRING, - user STRING, - time STRING, - request STRING, - status STRING, - size STRING, - referer STRING, - agent STRING) -ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.RegexSerDe' -STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation0.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation0.q deleted file mode 100644 index 4cb48664b602f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation0.q +++ /dev/null @@ -1,5 +0,0 @@ --- should fail: hive.join.cache.size accepts int type -desc src; - -set hive.conf.validation=true; -set hive.join.cache.size=test; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation1.q deleted file mode 100644 index 330aafd19858c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation1.q +++ /dev/null @@ -1,5 +0,0 @@ --- should fail: hive.map.aggr.hash.min.reduction accepts float type -desc src; - -set hive.conf.validation=true; -set hive.map.aggr.hash.min.reduction=false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation2.q deleted file mode 100644 index 579e9408b6c35..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation2.q +++ /dev/null @@ -1,5 +0,0 @@ --- should fail: hive.fetch.task.conversion accepts minimal or more -desc src; - -set hive.conf.validation=true; -set hive.fetch.task.conversion=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_table_property.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_table_property.q deleted file mode 100644 index d582aaeb386c2..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_table_property.q +++ /dev/null @@ -1,4 +0,0 @@ -create table testTable(col1 int, col2 int); - --- set a table property = null, it should be caught by the grammar -alter table testTable set tblproperties ('a'=); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns1.q deleted file mode 100644 index 25705dc3d5275..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns1.q +++ /dev/null @@ -1,2 +0,0 @@ -SHOW COLUMNS from shcol_test; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns2.q deleted file mode 100644 index c55b449a0b5ff..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns2.q +++ /dev/null @@ -1,2 +0,0 @@ -SHOW COLUMNS from shcol_test foo; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns3.q deleted file mode 100644 index 508a786609d8b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns3.q +++ /dev/null @@ -1,7 +0,0 @@ -CREATE DATABASE test_db; -USE test_db; -CREATE TABLE foo(a INT); - -use default; -SHOW COLUMNS from test_db.foo from test_db; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_create_table_does_not_exist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_create_table_does_not_exist.q deleted file mode 100644 index 83e5093aa1f21..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_create_table_does_not_exist.q +++ /dev/null @@ -1,2 +0,0 @@ -SHOW CREATE TABLE tmp_nonexist; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_create_table_index.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_create_table_index.q deleted file mode 100644 index 0dd0ef9a255bf..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_create_table_index.q +++ /dev/null @@ -1,6 +0,0 @@ -CREATE TABLE tmp_showcrt (key int, value string); -CREATE INDEX tmp_index on table tmp_showcrt(key) as 'compact' WITH DEFERRED REBUILD; -SHOW CREATE TABLE default__tmp_showcrt_tmp_index__; -DROP INDEX tmp_index on tmp_showcrt; -DROP TABLE tmp_showcrt; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_partitions1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_partitions1.q deleted file mode 100644 index 71f68c894f2a0..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_partitions1.q +++ /dev/null @@ -1 +0,0 @@ -SHOW PARTITIONS NonExistentTable; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tableproperties1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tableproperties1.q deleted file mode 100644 index 254a1d3a5ac3c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tableproperties1.q +++ /dev/null @@ -1 +0,0 @@ -SHOW TBLPROPERTIES NonExistentTable; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad1.q deleted file mode 100644 index 1bc94d6392c67..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad1.q +++ /dev/null @@ -1 +0,0 @@ -SHOW TABLES JOIN; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad2.q deleted file mode 100644 index 5e828b647ac38..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad2.q +++ /dev/null @@ -1 +0,0 @@ -SHOW TABLES FROM default LIKE a b; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad_db1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad_db1.q deleted file mode 100644 index d0141f6c291cb..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad_db1.q +++ /dev/null @@ -1 +0,0 @@ -SHOW TABLES FROM nonexistent; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad_db2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad_db2.q deleted file mode 100644 index ee0deba87a948..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad_db2.q +++ /dev/null @@ -1 +0,0 @@ -SHOW TABLES FROM nonexistent LIKE 'test'; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tablestatus.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tablestatus.q deleted file mode 100644 index 283b5836e27fa..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tablestatus.q +++ /dev/null @@ -1 +0,0 @@ -SHOW TABLE EXTENDED LIKE `srcpar*` PARTITION(ds='2008-04-08', hr=11); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tablestatus_not_existing_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tablestatus_not_existing_part.q deleted file mode 100644 index 242e165285547..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tablestatus_not_existing_part.q +++ /dev/null @@ -1 +0,0 @@ -SHOW TABLE EXTENDED LIKE `srcpart` PARTITION(ds='2008-14-08', hr=11); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/smb_bucketmapjoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/smb_bucketmapjoin.q deleted file mode 100644 index 880323c604b66..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/smb_bucketmapjoin.q +++ /dev/null @@ -1,23 +0,0 @@ -set hive.enforce.bucketing = true; -set hive.enforce.sorting = true; -set hive.exec.reducers.max = 1; - - -CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS; - - -CREATE TABLE smb_bucket4_2(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS; - -insert overwrite table smb_bucket4_1 -select * from src; - -insert overwrite table smb_bucket4_2 -select * from src; - -set hive.optimize.bucketmapjoin = true; -set hive.optimize.bucketmapjoin.sortedmerge = true; - -select /*+mapjoin(a)*/ * from smb_bucket4_1 a left outer join smb_bucket4_2 b on a.key = b.key; - - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/smb_mapjoin_14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/smb_mapjoin_14.q deleted file mode 100644 index 54bfba03d82d7..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/smb_mapjoin_14.q +++ /dev/null @@ -1,38 +0,0 @@ -set hive.enforce.bucketing = true; -set hive.enforce.sorting = true; -set hive.exec.reducers.max = 1; - -CREATE TABLE tbl1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; -CREATE TABLE tbl2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; - -insert overwrite table tbl1 -select * from src where key < 10; - -insert overwrite table tbl2 -select * from src where key < 10; - -set hive.optimize.bucketmapjoin = true; -set hive.optimize.bucketmapjoin.sortedmerge = true; -set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; - --- A join is being performed across different sub-queries, where a mapjoin is being performed in each of them. --- Each sub-query should be converted to a sort-merge join. --- A join followed by mapjoin is not allowed, so this query should fail. --- Once HIVE-3403 is in, this should be automatically converted to a sort-merge join without the hint -explain -select src1.key, src1.cnt1, src2.cnt1 from -( - select key, count(*) as cnt1 from - ( - select /*+mapjoin(a)*/ a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key - ) subq1 group by key -) src1 -join -( - select key, count(*) as cnt1 from - ( - select /*+mapjoin(a)*/ a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key - ) subq2 group by key -) src2 -on src1.key = src2.key -order by src1.key, src1.cnt1, src2.cnt1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sortmerge_mapjoin_mismatch_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sortmerge_mapjoin_mismatch_1.q deleted file mode 100644 index 7d11f450edfde..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sortmerge_mapjoin_mismatch_1.q +++ /dev/null @@ -1,28 +0,0 @@ -create table table_asc(key int, value string) CLUSTERED BY (key) SORTED BY (key asc) -INTO 1 BUCKETS STORED AS RCFILE; -create table table_desc(key int, value string) CLUSTERED BY (key) SORTED BY (key desc) -INTO 1 BUCKETS STORED AS RCFILE; - -set hive.enforce.bucketing = true; -set hive.enforce.sorting = true; - -insert overwrite table table_asc select key, value from src; -insert overwrite table table_desc select key, value from src; -set hive.optimize.bucketmapjoin = true; -set hive.optimize.bucketmapjoin.sortedmerge = true; -set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; - --- If the user asked for sort merge join to be enforced (by setting --- hive.enforce.sortmergebucketmapjoin to true), an error should be thrown, since --- one of the tables is in ascending order and the other is in descending order, --- and sort merge bucket mapjoin cannot be performed. In the default mode, the --- query would succeed, although a regular map-join would be performed instead of --- what the user asked. - -explain -select /*+mapjoin(a)*/ * from table_asc a join table_desc b on a.key = b.key; - -set hive.enforce.sortmergebucketmapjoin=true; - -explain -select /*+mapjoin(a)*/ * from table_asc a join table_desc b on a.key = b.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_out_of_range.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_out_of_range.q deleted file mode 100644 index 66af1fd7da682..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_out_of_range.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; - -select key from src tablesample(105 percent); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_wrong_format.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_wrong_format.q deleted file mode 100644 index f71cc4487910a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_wrong_format.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; - -select key from src tablesample(1 percent); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_wrong_format2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_wrong_format2.q deleted file mode 100644 index 1a13c0ff4cb28..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_wrong_format2.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; - -select key from src tablesample(1K); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q deleted file mode 100644 index 1b2872d3d7ed8..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q +++ /dev/null @@ -1,18 +0,0 @@ --- In this test, there is a dummy stats aggregator which throws an error when the --- method connect is called (as indicated by the parameter hive.test.dummystats.aggregator) --- If stats need not be reliable, the statement succeeds. However, if stats are supposed --- to be reliable (by setting hive.stats.reliable to true), the insert statement fails --- because stats cannot be collected for this statement - -create table tmptable(key string, value string); - -set hive.stats.dbclass=custom; -set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher; -set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.DummyStatsAggregator; -set hive.test.dummystats.aggregator=connect; - -set hive.stats.reliable=false; -INSERT OVERWRITE TABLE tmptable select * from src; - -set hive.stats.reliable=true; -INSERT OVERWRITE TABLE tmptable select * from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q deleted file mode 100644 index 0fa9ff6820371..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q +++ /dev/null @@ -1,16 +0,0 @@ --- In this test, the stats aggregator does not exists. --- If stats need not be reliable, the statement succeeds. However, if stats are supposed --- to be reliable (by setting hive.stats.reliable to true), the insert statement fails --- because stats cannot be collected for this statement - -create table tmptable(key string, value string); - -set hive.stats.dbclass=custom; -set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher; -set hive.stats.default.aggregator=""; - -set hive.stats.reliable=false; -INSERT OVERWRITE TABLE tmptable select * from src; - -set hive.stats.reliable=true; -INSERT OVERWRITE TABLE tmptable select * from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_noscan_non_native.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_noscan_non_native.q deleted file mode 100644 index bde66278360c5..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_noscan_non_native.q +++ /dev/null @@ -1,6 +0,0 @@ - -CREATE TABLE non_native1(key int, value string) -STORED BY 'org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler'; - --- we do not support analyze table ... noscan on non-native tables yet -analyze table non_native1 compute statistics noscan; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_autogether.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_autogether.q deleted file mode 100644 index 47a8148e0869c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_autogether.q +++ /dev/null @@ -1,31 +0,0 @@ -set datanucleus.cache.collections=false; -set hive.stats.autogather=false; -set hive.exec.dynamic.partition=true; -set hive.exec.dynamic.partition.mode=nonstrict; -set mapred.min.split.size=256; -set mapred.min.split.size.per.node=256; -set mapred.min.split.size.per.rack=256; -set mapred.max.split.size=256; - --- test analyze table ... compute statistics partialscan - --- 1. prepare data -CREATE table analyze_srcpart_partial_scan (key STRING, value STRING) -partitioned by (ds string, hr string) -stored as rcfile; -insert overwrite table analyze_srcpart_partial_scan partition (ds, hr) select * from srcpart where ds is not null; -describe formatted analyze_srcpart_partial_scan PARTITION(ds='2008-04-08',hr=11); - - --- 2. partialscan -explain -analyze table analyze_srcpart_partial_scan PARTITION(ds='2008-04-08',hr=11) compute statistics partialscan; -analyze table analyze_srcpart_partial_scan PARTITION(ds='2008-04-08',hr=11) compute statistics partialscan; - --- 3. confirm result -describe formatted analyze_srcpart_partial_scan PARTITION(ds='2008-04-08',hr=11); -describe formatted analyze_srcpart_partial_scan PARTITION(ds='2008-04-09',hr=11); -drop table analyze_srcpart_partial_scan; - - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_non_external.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_non_external.q deleted file mode 100644 index c206b8b5d765e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_non_external.q +++ /dev/null @@ -1,5 +0,0 @@ - -CREATE EXTERNAL TABLE external_table (key int, value string); - --- we do not support analyze table ... partialscan on EXTERNAL tables yet -analyze table external_table compute statistics partialscan; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_non_native.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_non_native.q deleted file mode 100644 index 8e02ced85e708..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_non_native.q +++ /dev/null @@ -1,6 +0,0 @@ - -CREATE TABLE non_native1(key int, value string) -STORED BY 'org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler'; - --- we do not support analyze table ... partialscan on non-native tables yet -analyze table non_native1 compute statistics partialscan; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partscan_norcfile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partscan_norcfile.q deleted file mode 100644 index 56d93d08aa697..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partscan_norcfile.q +++ /dev/null @@ -1,12 +0,0 @@ -set datanucleus.cache.collections=false; -set hive.stats.autogather=true; -set hive.exec.dynamic.partition=true; -set hive.exec.dynamic.partition.mode=nonstrict; - --- test analyze table ... compute statistics partialscan - -create table analyze_srcpart_partial_scan like srcpart; -insert overwrite table analyze_srcpart_partial_scan partition (ds, hr) select * from srcpart where ds is not null; -analyze table analyze_srcpart_partial_scan PARTITION(ds='2008-04-08',hr=11) compute statistics partialscan; - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_1.q deleted file mode 100644 index be7c4f72feb9b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_1.q +++ /dev/null @@ -1,18 +0,0 @@ --- In this test, there is a dummy stats publisher which throws an error when the --- method connect is called (as indicated by the parameter hive.test.dummystats.publisher) --- If stats need not be reliable, the statement succeeds. However, if stats are supposed --- to be reliable (by setting hive.stats.reliable to true), the insert statement fails --- because stats cannot be collected for this statement - -create table tmptable(key string, value string); - -set hive.stats.dbclass=custom; -set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher; -set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.DummyStatsAggregator; -set hive.test.dummystats.publisher=connect; - -set hive.stats.reliable=false; -INSERT OVERWRITE TABLE tmptable select * from src; - -set hive.stats.reliable=true; -INSERT OVERWRITE TABLE tmptable select * from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_2.q deleted file mode 100644 index 652afe7c5bfba..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_2.q +++ /dev/null @@ -1,16 +0,0 @@ --- In this test, the stats publisher does not exists. --- If stats need not be reliable, the statement succeeds. However, if stats are supposed --- to be reliable (by setting hive.stats.reliable to true), the insert statement fails --- because stats cannot be collected for this statement - -create table tmptable(key string, value string); - -set hive.stats.dbclass=custom; -set hive.stats.default.publisher=""; -set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.DummyStatsAggregator; - -set hive.stats.reliable=false; -INSERT OVERWRITE TABLE tmptable select * from src; - -set hive.stats.reliable=true; -INSERT OVERWRITE TABLE tmptable select * from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_join.q deleted file mode 100644 index d618ee28fdb2e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_join.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.mapred.mode=strict; - -SELECT * FROM src src1 JOIN src src2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_orderby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_orderby.q deleted file mode 100644 index 781cdbb05088b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_orderby.q +++ /dev/null @@ -1,7 +0,0 @@ -set hive.mapred.mode=strict; - -EXPLAIN -SELECT src.key, src.value from src order by src.key; - -SELECT src.key, src.value from src order by src.key; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_pruning.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_pruning.q deleted file mode 100644 index 270ab2f593ac2..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_pruning.q +++ /dev/null @@ -1,6 +0,0 @@ -set hive.mapred.mode=strict; - -EXPLAIN -SELECT count(1) FROM srcPART; - -SELECT count(1) FROM srcPART; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subq_insert.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subq_insert.q deleted file mode 100644 index 0bc9e24e4828c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subq_insert.q +++ /dev/null @@ -1,2 +0,0 @@ -EXPLAIN -SELECT * FROM (INSERT OVERWRITE TABLE src1 SELECT * FROM src ) y; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_exists_implicit_gby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_exists_implicit_gby.q deleted file mode 100644 index 9013df6f938dc..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_exists_implicit_gby.q +++ /dev/null @@ -1,10 +0,0 @@ - - -select * -from src b -where exists - (select count(*) - from src a - where b.value = a.value and a.key = b.key and a.value > 'val_9' - ) -; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_groupby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_groupby.q deleted file mode 100644 index a9bc6ee6a38cb..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_groupby.q +++ /dev/null @@ -1,5 +0,0 @@ - - -select count(*) -from src -group by src.key in (select key from src s1 where s1.key > '9') \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_select.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_select.q deleted file mode 100644 index 1365389cb269a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_select.q +++ /dev/null @@ -1,6 +0,0 @@ - - - -select src.key in (select key from src s1 where s1.key > '9') -from src -; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_multiple_cols_in_select.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_multiple_cols_in_select.q deleted file mode 100644 index 6805c5b16b0f4..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_multiple_cols_in_select.q +++ /dev/null @@ -1,7 +0,0 @@ - - -explain - select * -from src -where src.key in (select * from src s1 where s1.key > '9') -; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_nested_subquery.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_nested_subquery.q deleted file mode 100644 index e8c41e6b17ae7..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_nested_subquery.q +++ /dev/null @@ -1,18 +0,0 @@ - - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - -select * -from part x -where x.p_name in (select y.p_name from part y where exists (select z.p_name from part z where y.p_name = z.p_name)) -; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_notexists_implicit_gby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_notexists_implicit_gby.q deleted file mode 100644 index 852b2953ff463..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_notexists_implicit_gby.q +++ /dev/null @@ -1,10 +0,0 @@ - - -select * -from src b -where not exists - (select sum(1) - from src a - where b.value = a.value and a.key = b.key and a.value > 'val_9' - ) -; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_shared_alias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_shared_alias.q deleted file mode 100644 index d442f077c0707..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_shared_alias.q +++ /dev/null @@ -1,6 +0,0 @@ - - -select * -from src -where src.key in (select key from src where key > '9') -; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_subquery_chain.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_subquery_chain.q deleted file mode 100644 index 8ea94c5fc6d76..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_subquery_chain.q +++ /dev/null @@ -1,6 +0,0 @@ - -explain -select * -from src -where src.key in (select key from src) in (select key from src) -; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_unqual_corr_expr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_unqual_corr_expr.q deleted file mode 100644 index 99ff9ca703835..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_unqual_corr_expr.q +++ /dev/null @@ -1,6 +0,0 @@ - - -select * -from src -where key in (select key from src) -; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_windowing_corr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_windowing_corr.q deleted file mode 100644 index 105d3d22d9d2b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_windowing_corr.q +++ /dev/null @@ -1,26 +0,0 @@ -DROP TABLE part; - --- data setup -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - -LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; - - --- corr and windowing -select p_mfgr, p_name, p_size -from part a -where a.p_size in - (select first_value(p_size) over(partition by p_mfgr order by p_size) - from part b - where a.p_brand = b.p_brand) -; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_with_or_cond.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_with_or_cond.q deleted file mode 100644 index c2c322178f386..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_with_or_cond.q +++ /dev/null @@ -1,5 +0,0 @@ - -select count(*) -from src -where src.key in (select key from src s1 where s1.key > '9') or src.value is not null -; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/touch1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/touch1.q deleted file mode 100644 index 9efbba0082b6f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/touch1.q +++ /dev/null @@ -1 +0,0 @@ -ALTER TABLE srcpart TOUCH PARTITION (ds='2008-04-08', hr='13'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/touch2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/touch2.q deleted file mode 100644 index 923a171e0482e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/touch2.q +++ /dev/null @@ -1 +0,0 @@ -ALTER TABLE src TOUCH PARTITION (ds='2008-04-08', hr='12'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_bucketed_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_bucketed_column.q deleted file mode 100644 index e53665695a39d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_bucketed_column.q +++ /dev/null @@ -1,7 +0,0 @@ --- Tests truncating a bucketed column - -CREATE TABLE test_tab (key STRING, value STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS RCFILE; - -INSERT OVERWRITE TABLE test_tab SELECT * FROM src; - -TRUNCATE TABLE test_tab COLUMNS (key); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_indexed_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_indexed_table.q deleted file mode 100644 index 13f32c8968a16..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_indexed_table.q +++ /dev/null @@ -1,9 +0,0 @@ --- Tests truncating a column from an indexed table - -CREATE TABLE test_tab (key STRING, value STRING) STORED AS RCFILE; - -INSERT OVERWRITE TABLE test_tab SELECT * FROM src; - -CREATE INDEX test_tab_index ON TABLE test_tab (key) as 'COMPACT' WITH DEFERRED REBUILD; - -TRUNCATE TABLE test_tab COLUMNS (value); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_list_bucketing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_list_bucketing.q deleted file mode 100644 index 0ece6007f7b61..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_list_bucketing.q +++ /dev/null @@ -1,14 +0,0 @@ -set hive.mapred.supports.subdirectories=true; -set mapred.input.dir.recursive=true; - --- Tests truncating a column on which a table is list bucketed - -CREATE TABLE test_tab (key STRING, value STRING) STORED AS RCFILE; - -ALTER TABLE test_tab -SKEWED BY (key) ON ("484") -STORED AS DIRECTORIES; - -INSERT OVERWRITE TABLE test_tab SELECT * FROM src; - -TRUNCATE TABLE test_tab COLUMNS (key); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_seqfile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_seqfile.q deleted file mode 100644 index 903540dae8985..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_seqfile.q +++ /dev/null @@ -1,7 +0,0 @@ --- Tests truncating a column from a table stored as a sequence file - -CREATE TABLE test_tab (key STRING, value STRING) STORED AS SEQUENCEFILE; - -INSERT OVERWRITE TABLE test_tab SELECT * FROM src; - -TRUNCATE TABLE test_tab COLUMNS (key); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_nonexistant_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_nonexistant_column.q deleted file mode 100644 index 5509552811b09..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_nonexistant_column.q +++ /dev/null @@ -1,7 +0,0 @@ --- Tests attempting to truncate a column in a table that doesn't exist - -CREATE TABLE test_tab (key STRING, value STRING) STORED AS RCFILE; - -INSERT OVERWRITE TABLE test_tab SELECT * FROM src; - -TRUNCATE TABLE test_tab COLUMNS (doesnt_exist); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_partition_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_partition_column.q deleted file mode 100644 index 134743ac13a5d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_partition_column.q +++ /dev/null @@ -1,7 +0,0 @@ --- Tests truncating a partition column - -CREATE TABLE test_tab (key STRING, value STRING) PARTITIONED BY (part STRING) STORED AS RCFILE; - -INSERT OVERWRITE TABLE test_tab PARTITION (part = '1') SELECT * FROM src; - -TRUNCATE TABLE test_tab COLUMNS (part); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_partition_column2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_partition_column2.q deleted file mode 100644 index 47635208a7810..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_partition_column2.q +++ /dev/null @@ -1,7 +0,0 @@ --- Tests truncating a partition column - -CREATE TABLE test_tab (key STRING, value STRING) PARTITIONED BY (part STRING) STORED AS RCFILE; - -INSERT OVERWRITE TABLE test_tab PARTITION (part = '1') SELECT * FROM src; - -TRUNCATE TABLE test_tab PARTITION (part = '1') COLUMNS (part); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure1.q deleted file mode 100644 index f6cfa44bbb126..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure1.q +++ /dev/null @@ -1,2 +0,0 @@ --- partition spec for non-partitioned table -TRUNCATE TABLE src partition (ds='2008-04-08', hr='11'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure2.q deleted file mode 100644 index 1137d893eb0e1..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure2.q +++ /dev/null @@ -1,2 +0,0 @@ --- full partition spec for not existing partition -TRUNCATE TABLE srcpart partition (ds='2012-12-17', hr='15'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure3.q deleted file mode 100644 index c5cf58775b309..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure3.q +++ /dev/null @@ -1,4 +0,0 @@ -create external table external1 (a int, b int) partitioned by (ds string); - --- trucate for non-managed table -TRUNCATE TABLE external1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure4.q deleted file mode 100644 index a7f1e92d55980..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure4.q +++ /dev/null @@ -1,5 +0,0 @@ -CREATE TABLE non_native(key int, value string) -STORED BY 'org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler'; - --- trucate for non-native table -TRUNCATE TABLE non_native; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udaf_invalid_place.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udaf_invalid_place.q deleted file mode 100644 index f37ce72ae4197..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udaf_invalid_place.q +++ /dev/null @@ -1 +0,0 @@ -select distinct key, sum(key) from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_array_contains_wrong1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_array_contains_wrong1.q deleted file mode 100644 index c2a132d4db05e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_array_contains_wrong1.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid first argument -SELECT array_contains(1, 2) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_array_contains_wrong2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_array_contains_wrong2.q deleted file mode 100644 index 36f85d34a6e0b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_array_contains_wrong2.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid second argument -SELECT array_contains(array(1, 2, 3), '2') FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_assert_true.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_assert_true.q deleted file mode 100644 index 73b3f9654f1c3..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_assert_true.q +++ /dev/null @@ -1,7 +0,0 @@ -DESCRIBE FUNCTION ASSERT_TRUE; - -EXPLAIN SELECT ASSERT_TRUE(x > 0) FROM src LATERAL VIEW EXPLODE(ARRAY(1, 2)) a AS x LIMIT 2; -SELECT ASSERT_TRUE(x > 0) FROM src LATERAL VIEW EXPLODE(ARRAY(1, 2)) a AS x LIMIT 2; - -EXPLAIN SELECT ASSERT_TRUE(x < 2) FROM src LATERAL VIEW EXPLODE(ARRAY(1, 2)) a AS x LIMIT 2; -SELECT ASSERT_TRUE(x < 2) FROM src LATERAL VIEW EXPLODE(ARRAY(1, 2)) a AS x LIMIT 2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_assert_true2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_assert_true2.q deleted file mode 100644 index 4b62220764bb9..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_assert_true2.q +++ /dev/null @@ -1,2 +0,0 @@ -EXPLAIN SELECT 1 + ASSERT_TRUE(x < 2) FROM src LATERAL VIEW EXPLODE(ARRAY(1, 2)) a AS x LIMIT 2; -SELECT 1 + ASSERT_TRUE(x < 2) FROM src LATERAL VIEW EXPLODE(ARRAY(1, 2)) a AS x LIMIT 2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_coalesce.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_coalesce.q deleted file mode 100644 index 7405e387caf70..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_coalesce.q +++ /dev/null @@ -1 +0,0 @@ -SELECT COALESCE(array('a', 'b'), '2.0') FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong1.q deleted file mode 100644 index 8c2017bc636cf..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong1.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument number -SELECT concat_ws('-') FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong2.q deleted file mode 100644 index c49e7868bbb5e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong2.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument type -SELECT concat_ws('[]', array(100, 200, 50)) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong3.q deleted file mode 100644 index 72b86271f5ea6..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong3.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument type -SELECT concat_ws(1234, array('www', 'facebook', 'com')) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_elt_wrong_args_len.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_elt_wrong_args_len.q deleted file mode 100644 index fbe4902d644cc..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_elt_wrong_args_len.q +++ /dev/null @@ -1 +0,0 @@ -SELECT elt(3) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_elt_wrong_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_elt_wrong_type.q deleted file mode 100644 index bb1fdbf789e3e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_elt_wrong_type.q +++ /dev/null @@ -1,3 +0,0 @@ -FROM src_thrift -SELECT elt(1, src_thrift.lintstring) -WHERE src_thrift.lintstring IS NOT NULL; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_field_wrong_args_len.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_field_wrong_args_len.q deleted file mode 100644 index 9703c82d8a4d9..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_field_wrong_args_len.q +++ /dev/null @@ -1 +0,0 @@ -SELECT field(3) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_field_wrong_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_field_wrong_type.q deleted file mode 100644 index 61b2cd06496e2..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_field_wrong_type.q +++ /dev/null @@ -1,3 +0,0 @@ -FROM src_thrift -SELECT field(1, src_thrift.lintstring) -WHERE src_thrift.lintstring IS NOT NULL; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong1.q deleted file mode 100644 index 18c985c606849..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong1.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument length -SELECT format_number(12332.123456) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong2.q deleted file mode 100644 index 7959c20b28e56..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong2.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument length -SELECT format_number(12332.123456, 2, 3) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong3.q deleted file mode 100644 index 7d90ef86da7b7..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong3.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument(second argument should be >= 0) -SELECT format_number(12332.123456, -4) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong4.q deleted file mode 100644 index e545f4aa1420c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong4.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument type -SELECT format_number(12332.123456, 4.01) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong5.q deleted file mode 100644 index a6f71778f143b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong5.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument type -SELECT format_number(array(12332.123456, 321.23), 5) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong6.q deleted file mode 100644 index e5b11b9b71ee7..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong6.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument type -SELECT format_number(12332.123456, "4") FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong7.q deleted file mode 100644 index aa4a3a44751c4..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong7.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument type(format_number returns the result as a string) -SELECT format_number(format_number(12332.123456, 4), 2) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_function_does_not_implement_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_function_does_not_implement_udf.q deleted file mode 100644 index 21ca6e7d36255..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_function_does_not_implement_udf.q +++ /dev/null @@ -1 +0,0 @@ -CREATE TEMPORARY FUNCTION moo AS 'org.apache.hadoop.hive.ql.Driver'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_if_not_bool.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_if_not_bool.q deleted file mode 100644 index 74458d0c3db2c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_if_not_bool.q +++ /dev/null @@ -1 +0,0 @@ -SELECT IF('STRING', 1, 1) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_if_wrong_args_len.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_if_wrong_args_len.q deleted file mode 100644 index ad19364c33076..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_if_wrong_args_len.q +++ /dev/null @@ -1 +0,0 @@ -SELECT IF(TRUE, 1) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_in.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_in.q deleted file mode 100644 index ce9ce54fac680..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_in.q +++ /dev/null @@ -1 +0,0 @@ -SELECT 3 IN (array(1,2,3)) FROM src; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_instr_wrong_args_len.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_instr_wrong_args_len.q deleted file mode 100644 index ac8253fb1e941..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_instr_wrong_args_len.q +++ /dev/null @@ -1 +0,0 @@ -SELECT instr('abcd') FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_instr_wrong_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_instr_wrong_type.q deleted file mode 100644 index 9ac3ed6614897..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_instr_wrong_type.q +++ /dev/null @@ -1,3 +0,0 @@ -FROM src_thrift -SELECT instr('abcd', src_thrift.lintstring) -WHERE src_thrift.lintstring IS NOT NULL; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_invalid.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_invalid.q deleted file mode 100644 index 68050fd95cd22..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_invalid.q +++ /dev/null @@ -1 +0,0 @@ -select default.nonexistfunc() from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_local_resource.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_local_resource.q deleted file mode 100644 index bcfa217737e33..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_local_resource.q +++ /dev/null @@ -1 +0,0 @@ -create function lookup as 'org.apache.hadoop.hive.ql.udf.UDFFileLookup' using file '../../data/files/sales.txt'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_locate_wrong_args_len.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_locate_wrong_args_len.q deleted file mode 100644 index ca7caad54d646..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_locate_wrong_args_len.q +++ /dev/null @@ -1 +0,0 @@ -SELECT locate('a', 'b', 1, 2) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_locate_wrong_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_locate_wrong_type.q deleted file mode 100644 index 4bbf79a310b01..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_locate_wrong_type.q +++ /dev/null @@ -1,3 +0,0 @@ -FROM src_thrift -SELECT locate('abcd', src_thrift.lintstring) -WHERE src_thrift.lintstring IS NOT NULL; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_keys_arg_num.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_keys_arg_num.q deleted file mode 100644 index ebb6c2ab418eb..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_keys_arg_num.q +++ /dev/null @@ -1 +0,0 @@ -SELECT map_keys(map("a", "1"), map("b", "2")) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_keys_arg_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_keys_arg_type.q deleted file mode 100644 index 0757d1494f3cd..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_keys_arg_type.q +++ /dev/null @@ -1 +0,0 @@ -SELECT map_keys(array(1, 2, 3)) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_values_arg_num.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_values_arg_num.q deleted file mode 100644 index c97476a1263ee..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_values_arg_num.q +++ /dev/null @@ -1 +0,0 @@ -SELECT map_values(map("a", "1"), map("b", "2")) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_values_arg_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_values_arg_type.q deleted file mode 100644 index cc060ea0f0ec7..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_values_arg_type.q +++ /dev/null @@ -1 +0,0 @@ -SELECT map_values(array(1, 2, 3, 4)) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_max.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_max.q deleted file mode 100644 index 7282e07596032..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_max.q +++ /dev/null @@ -1,2 +0,0 @@ -SELECT max(map("key", key, "value", value)) -FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_min.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_min.q deleted file mode 100644 index b9528fa6dafef..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_min.q +++ /dev/null @@ -1,2 +0,0 @@ -SELECT min(map("key", key, "value", value)) -FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_nonexistent_resource.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_nonexistent_resource.q deleted file mode 100644 index d37665dde69bc..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_nonexistent_resource.q +++ /dev/null @@ -1 +0,0 @@ -create function lookup as 'org.apache.hadoop.hive.ql.udf.UDFFileLookup' using file 'nonexistent_file.txt'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong1.q deleted file mode 100644 index 88ca4fefc3055..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong1.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument length -SELECT printf() FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong2.q deleted file mode 100644 index 01ed2ffcf017f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong2.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument type -SELECT printf(100) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong3.q deleted file mode 100644 index 71f118b8dc0dd..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong3.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument type -SELECT printf("Hello World %s", array("invalid", "argument")) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong4.q deleted file mode 100644 index 71f118b8dc0dd..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong4.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument type -SELECT printf("Hello World %s", array("invalid", "argument")) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_qualified_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_qualified_name.q deleted file mode 100644 index 476dfa21a237f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_qualified_name.q +++ /dev/null @@ -1 +0,0 @@ -create temporary function default.myfunc as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFSum'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_reflect_neg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_reflect_neg.q deleted file mode 100644 index 67efb64505d9e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_reflect_neg.q +++ /dev/null @@ -1,9 +0,0 @@ -SELECT reflect("java.lang.StringClassThatDoesNotExist", "valueOf", 1), - reflect("java.lang.String", "methodThatDoesNotExist"), - reflect("java.lang.Math", "max", "overloadthatdoesnotexist", 3), - reflect("java.lang.Math", "min", 2, 3), - reflect("java.lang.Math", "round", 2.5), - reflect("java.lang.Math", "exp", 1.0), - reflect("java.lang.Math", "floor", 1.9) -FROM src LIMIT 1; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_size_wrong_args_len.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_size_wrong_args_len.q deleted file mode 100644 index c628ff8aa1972..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_size_wrong_args_len.q +++ /dev/null @@ -1,5 +0,0 @@ -FROM src_thrift -SELECT size(src_thrift.lint, src_thrift.lintstring), - size() -WHERE src_thrift.lint IS NOT NULL - AND NOT (src_thrift.mstringstring IS NULL) LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_size_wrong_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_size_wrong_type.q deleted file mode 100644 index 16695f6adc3f4..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_size_wrong_type.q +++ /dev/null @@ -1 +0,0 @@ -SELECT SIZE('wrong type: string') FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong1.q deleted file mode 100644 index 9954f4ab4d3c5..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong1.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument number -SELECT sort_array(array(2, 5, 4), 3) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong2.q deleted file mode 100644 index 32c2645519497..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong2.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument type -SELECT sort_array("Invalid") FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong3.q deleted file mode 100644 index 034de06b8e391..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong3.q +++ /dev/null @@ -1,2 +0,0 @@ --- invalid argument type -SELECT sort_array(array(array(10, 20), array(5, 15), array(3, 13))) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_test_error.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_test_error.q deleted file mode 100644 index 846f87c2e51b1..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_test_error.q +++ /dev/null @@ -1,3 +0,0 @@ -CREATE TEMPORARY FUNCTION test_error AS 'org.apache.hadoop.hive.ql.udf.UDFTestErrorOnFalse'; - -SELECT test_error(key < 125 OR key > 130) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_test_error_reduce.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_test_error_reduce.q deleted file mode 100644 index b1a06f2a07af3..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_test_error_reduce.q +++ /dev/null @@ -1,11 +0,0 @@ -CREATE TEMPORARY FUNCTION test_error AS 'org.apache.hadoop.hive.ql.udf.UDFTestErrorOnFalse'; - - -SELECT test_error(key < 125 OR key > 130) -FROM ( - SELECT * - FROM src - DISTRIBUTE BY rand() -) map_output; - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong.q deleted file mode 100644 index d4d2d2e48517f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong.q +++ /dev/null @@ -1,6 +0,0 @@ -SELECT CASE - WHEN TRUE THEN 2 - WHEN '1' THEN 4 - ELSE 5 - END -FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported1.q deleted file mode 100644 index 942ae5d8315f5..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported1.q +++ /dev/null @@ -1 +0,0 @@ -SELECT explode(map(1,'one',2,'two',3,'three')) as (myKey,myVal) FROM src GROUP BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported2.q deleted file mode 100644 index 00d359a75ce0c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported2.q +++ /dev/null @@ -1 +0,0 @@ -SELECT explode(map(1,'one',2,'two',3,'three')) as (myKey,myVal,myVal2) FROM src; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported3.q deleted file mode 100644 index 51df8fa862e1a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported3.q +++ /dev/null @@ -1 +0,0 @@ -select explode(array(1),array(2)) as myCol from src; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported4.q deleted file mode 100644 index ae8dff7bad8d8..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported4.q +++ /dev/null @@ -1 +0,0 @@ -SELECT explode(null) as myNull FROM src GROUP BY key; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_invalid_place.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_invalid_place.q deleted file mode 100644 index ab84a801e9ed6..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_invalid_place.q +++ /dev/null @@ -1 +0,0 @@ -select distinct key, explode(key) from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_not_supported1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_not_supported1.q deleted file mode 100644 index 04e98d52c548a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_not_supported1.q +++ /dev/null @@ -1 +0,0 @@ -SELECT explode(array(1,2,3)) as myCol, key FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_not_supported3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_not_supported3.q deleted file mode 100644 index f4fe0dde3e62c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_not_supported3.q +++ /dev/null @@ -1 +0,0 @@ -SELECT explode(array(1,2,3)) as myCol FROM src GROUP BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union2.q deleted file mode 100644 index 38db488eaf68e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union2.q +++ /dev/null @@ -1,13 +0,0 @@ - - -create table if not exists union2_t1(r string, c string, v array); -create table if not exists union2_t2(s string, c string, v string); - -explain -SELECT s.r, s.c, sum(s.v) -FROM ( - SELECT a.r AS r, a.c AS c, a.v AS v FROM union2_t1 a - UNION ALL - SELECT b.s AS r, b.c AS c, 0 + b.v AS v FROM union2_t2 b -) s -GROUP BY s.r, s.c; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union22.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union22.q deleted file mode 100644 index 72f3314bdac96..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union22.q +++ /dev/null @@ -1,26 +0,0 @@ -create table dst_union22(k1 string, k2 string, k3 string, k4 string) partitioned by (ds string); -create table dst_union22_delta(k0 string, k1 string, k2 string, k3 string, k4 string, k5 string) partitioned by (ds string); - -insert overwrite table dst_union22 partition (ds='1') -select key, value, key , value from src; - -insert overwrite table dst_union22_delta partition (ds='1') -select key, key, value, key, value, value from src; - -set hive.merge.mapfiles=false; - --- Union followed by Mapjoin is not supported. --- The same query would work without the hint --- Note that there is a positive test with the same name in clientpositive -explain extended -insert overwrite table dst_union22 partition (ds='2') -select * from -( -select k1 as k1, k2 as k2, k3 as k3, k4 as k4 from dst_union22_delta where ds = '1' and k0 <= 50 -union all -select /*+ MAPJOIN(b) */ a.k1 as k1, a.k2 as k2, b.k3 as k3, b.k4 as k4 -from dst_union22 a left outer join (select * from dst_union22_delta where ds = '1' and k0 > 50) b on -a.k1 = b.k1 and a.ds='1' -where a.k1 > 20 -) -subq; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union3.q deleted file mode 100644 index ce657478c150d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union3.q +++ /dev/null @@ -1,5 +0,0 @@ --- Ensure that UNION ALL columns are in the correct order on both sides --- Ensure that the appropriate error message is propagated -CREATE TABLE IF NOT EXISTS union3 (bar int, baz int); -SELECT * FROM ( SELECT f.bar, f.baz FROM union3 f UNION ALL SELECT b.baz, b.bar FROM union3 b ) c; -DROP TABLE union3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin.q deleted file mode 100644 index d6a19c397d80d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin.q +++ /dev/null @@ -1,3 +0,0 @@ -FROM UNIQUEJOIN (SELECT src.key from src WHERE src.key<4) a (a.key), PRESERVE src b(b.key) -SELECT a.key, b.key; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin2.q deleted file mode 100644 index 6e9a08251407d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin2.q +++ /dev/null @@ -1,3 +0,0 @@ -FROM UNIQUEJOIN src a (a.key), PRESERVE src b (b.key, b.val) -SELECT a.key, b.key; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin3.q deleted file mode 100644 index 89a8f1b2aaa84..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin3.q +++ /dev/null @@ -1,3 +0,0 @@ -FROM UNIQUEJOIN src a (a.key), PRESERVE src b (b.key) JOIN src c ON c.key -SELECT a.key; - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/unset_table_property.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/unset_table_property.q deleted file mode 100644 index 7a24e652b46f2..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/unset_table_property.q +++ /dev/null @@ -1,6 +0,0 @@ -CREATE TABLE testTable(col1 INT, col2 INT); -ALTER TABLE testTable SET TBLPROPERTIES ('a'='1', 'c'='3'); -SHOW TBLPROPERTIES testTable; - --- unset a subset of the properties and some non-existed properties without if exists -ALTER TABLE testTable UNSET TBLPROPERTIES ('c', 'x', 'y', 'z'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/unset_view_property.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/unset_view_property.q deleted file mode 100644 index 11131006e998a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/unset_view_property.q +++ /dev/null @@ -1,6 +0,0 @@ -CREATE VIEW testView AS SELECT value FROM src WHERE key=86; -ALTER VIEW testView SET TBLPROPERTIES ('propA'='100', 'propB'='200'); -SHOW TBLPROPERTIES testView; - --- unset a subset of the properties and some non-existed properties without if exists -ALTER VIEW testView UNSET TBLPROPERTIES ('propB', 'propX', 'propY', 'propZ'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_invalid_udaf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_invalid_udaf.q deleted file mode 100644 index c5b593e4bb556..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_invalid_udaf.q +++ /dev/null @@ -1 +0,0 @@ -select nonexistfunc(key) over () from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_leadlag_in_udaf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_leadlag_in_udaf.q deleted file mode 100644 index b54b7a532176f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_leadlag_in_udaf.q +++ /dev/null @@ -1,15 +0,0 @@ -DROP TABLE part; - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - -select sum(lead(p_retailprice,1)) as s1 from part; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_neg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_neg.q deleted file mode 100644 index 15f8fae292bbb..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_neg.q +++ /dev/null @@ -1,26 +0,0 @@ -DROP TABLE IF EXISTS part; - --- data setup -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - -LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; - - -select p_mfgr, p_name, p_size, -min(p_retailprice), -rank() over(distribute by p_mfgr sort by p_name)as r, -dense_rank() over(distribute by p_mfgr sort by p_name) as dr, -p_size, p_size - lag(p_size,-1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz -from part -group by p_mfgr, p_name, p_size -; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_over.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_over.q deleted file mode 100644 index 3ca1104b0158d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_over.q +++ /dev/null @@ -1,17 +0,0 @@ -DROP TABLE part; - -CREATE TABLE part( - p_partkey INT, - p_name STRING, - p_mfgr STRING, - p_brand STRING, - p_type STRING, - p_size INT, - p_container STRING, - p_retailprice DOUBLE, - p_comment STRING -); - -select p_mfgr, -lead(p_retailprice,1) as s1 -from part; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/wrong_column_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/wrong_column_type.q deleted file mode 100644 index 490f0c3b4d110..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/wrong_column_type.q +++ /dev/null @@ -1,4 +0,0 @@ -CREATE TABLE dest1(a float); - -INSERT OVERWRITE TABLE dest1 -SELECT array(1.0,2.0) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/ambiguous_join_col.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/ambiguous_join_col.q deleted file mode 100644 index e70aae46275b0..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/ambiguous_join_col.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src src1 JOIN src src2 ON src1.key = src2.key -INSERT OVERWRITE TABLE dest1 SELECT key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/duplicate_alias.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/duplicate_alias.q deleted file mode 100644 index 5fd22460c037a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/duplicate_alias.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src a JOIN src a ON (a.key = a.key) -INSERT OVERWRITE TABLE dest1 SELECT a.key, a.value diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/garbage.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/garbage.q deleted file mode 100644 index 6c8c751f21c31..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/garbage.q +++ /dev/null @@ -1 +0,0 @@ -this is totally garbage SELECT src.key WHERE a lot of garbage diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/insert_wrong_number_columns.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/insert_wrong_number_columns.q deleted file mode 100644 index aadfbde33836a..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/insert_wrong_number_columns.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value, 1 WHERE src.key < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_create_table.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_create_table.q deleted file mode 100644 index 899bbd368b185..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_create_table.q +++ /dev/null @@ -1,4 +0,0 @@ -CREATE TABLE mytable ( - a INT - b STRING -); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_dot.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_dot.q deleted file mode 100644 index 36b9bd2a3b980..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_dot.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT src.value.member WHERE src.key < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_function_param2.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_function_param2.q deleted file mode 100644 index 3543449b8870b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_function_param2.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT substr('1234', 'abc'), src.value WHERE src.key < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_index.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_index.q deleted file mode 100644 index 146bc5dc9f3b0..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_index.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT src.key[0], src.value diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_list_index.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_list_index.q deleted file mode 100644 index c40f079f60aaa..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_list_index.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src_thrift -INSERT OVERWRITE TABLE dest1 SELECT src_thrift.lint[0], src_thrift.lstring['abc'] diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_list_index2.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_list_index2.q deleted file mode 100644 index 99d0b3d4162ac..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_list_index2.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src_thrift -INSERT OVERWRITE TABLE dest1 SELECT src_thrift.lint[0], src_thrift.lstring[1 + 2] diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_map_index.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_map_index.q deleted file mode 100644 index c2b9eab61b80e..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_map_index.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src_thrift -INSERT OVERWRITE TABLE dest1 SELECT src_thrift.lint[0], src_thrift.mstringstring[0] diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_map_index2.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_map_index2.q deleted file mode 100644 index 5828f0709f53b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_map_index2.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src_thrift -INSERT OVERWRITE TABLE dest1 SELECT src_thrift.lint[0], src_thrift.mstringstring[concat('abc', 'abc')] diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_select.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_select.q deleted file mode 100644 index fd1298577be8c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_select.q +++ /dev/null @@ -1,4 +0,0 @@ -SELECT - trim(trim(a)) - trim(b) -FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/macro_reserved_word.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/macro_reserved_word.q deleted file mode 100644 index 359eb9de93bac..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/macro_reserved_word.q +++ /dev/null @@ -1 +0,0 @@ -CREATE TEMPORARY MACRO DOUBLE (x DOUBLE) 1.0 / (1.0 + EXP(-x)); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/missing_overwrite.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/missing_overwrite.q deleted file mode 100644 index 1bfeee382ea37..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/missing_overwrite.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT TABLE dest1 SELECT '1234', src.value WHERE src.key < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/nonkey_groupby.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/nonkey_groupby.q deleted file mode 100644 index ad0f4415cbd8c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/nonkey_groupby.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234', src.value WHERE src.key < 100 group by src.key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/quoted_string.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/quoted_string.q deleted file mode 100644 index 0252a9e11cdf5..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/quoted_string.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234", src.value WHERE src.key < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column1.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column1.q deleted file mode 100644 index 429cead63beb1..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column1.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234', src.dummycol WHERE src.key < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column2.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column2.q deleted file mode 100644 index 3767dc4e65021..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column2.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234', src.value WHERE src.dummykey < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column3.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column3.q deleted file mode 100644 index 2fc5f490f118d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column3.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234', src.value WHERE src.key < 100 group by src.dummycol diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column4.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column4.q deleted file mode 100644 index 8ad8dd12e46e2..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column4.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234', src.value WHERE src.key < 100 group by dummysrc.key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column5.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column5.q deleted file mode 100644 index 766b0e5255fea..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column5.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234', src.value WHERE dummysrc.key < 100 group by src.key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column6.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column6.q deleted file mode 100644 index bb76c28623484..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column6.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234', dummysrc.value WHERE src.key < 100 group by src.key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function1.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function1.q deleted file mode 100644 index d8ff6325b95f2..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function1.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234', dummyfn(src.value, 10) WHERE src.key < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function2.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function2.q deleted file mode 100644 index f7d255934db5c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function2.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234', src.value WHERE anotherdummyfn('abc', src.key) + 10 < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function3.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function3.q deleted file mode 100644 index 87d4edc987865..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function3.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234', src.value WHERE anotherdummyfn('abc', src.key) + 10 < 100 group by src.key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function4.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function4.q deleted file mode 100644 index cfe70e4f2fdc4..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function4.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234', dummyfn(src.key) WHERE src.key < 100 group by src.key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_table1.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_table1.q deleted file mode 100644 index 585ef6d7f2db1..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_table1.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM dummySrc -INSERT OVERWRITE TABLE dest1 SELECT '1234', src.value WHERE src.key < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_table2.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_table2.q deleted file mode 100644 index 2c69c16be5905..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_table2.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dummyDest SELECT '1234', src.value WHERE src.key < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/wrong_distinct1.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/wrong_distinct1.q deleted file mode 100755 index d92c3bb8df4b0..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/wrong_distinct1.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT DISTINCT src.key, substr(src.value,4,1) GROUP BY src.key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/wrong_distinct2.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/wrong_distinct2.q deleted file mode 100755 index 53fb550b3d111..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/negative/wrong_distinct2.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT src.key, DISTINCT substr(src.value,4,1) GROUP BY src.key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/case_sensitivity.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/case_sensitivity.q deleted file mode 100644 index d7f7371507662..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/case_sensitivity.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM SRC_THRIFT -INSERT OVERWRITE TABLE dest1 SELECT src_Thrift.LINT[1], src_thrift.lintstring[0].MYSTRING where src_thrift.liNT[0] > 0 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/cast1.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/cast1.q deleted file mode 100644 index 6269c6a4e76f9..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/cast1.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -SELECT 3 + 2, 3.0 + 2, 3 + 2.0, 3.0 + 2.0, 3 + CAST(2.0 AS INT), CAST(1 AS BOOLEAN), CAST(TRUE AS INT) WHERE src.key = 86 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby1.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby1.q deleted file mode 100755 index 96b29b05cc7ad..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby1.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby2.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby2.q deleted file mode 100755 index d741eb60b6bb1..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby2.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))) GROUP BY substr(src.key,1,1) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby3.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby3.q deleted file mode 100755 index 03b1248a11cb6..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby3.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -SELECT sum(substr(src.value,5)), avg(substr(src.value,5)), avg(DISTINCT substr(src.value,5)), max(substr(src.value,5)), min(substr(src.value,5)) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby4.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby4.q deleted file mode 100755 index 85271a9caf6e3..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby4.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -SELECT substr(src.key,1,1) GROUP BY substr(src.key,1,1) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby5.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby5.q deleted file mode 100755 index ebd65b3069722..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby5.q +++ /dev/null @@ -1,4 +0,0 @@ - -SELECT src.key, sum(substr(src.value,5)) -FROM src -GROUP BY src.key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby6.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby6.q deleted file mode 100755 index 80654f2a9ce6b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby6.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -SELECT DISTINCT substr(src.value,5,1) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input1.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input1.q deleted file mode 100644 index fdd290d6b136b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input1.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT src.key, src.value WHERE src.key < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input2.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input2.q deleted file mode 100644 index 4e1612ea972e9..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input2.q +++ /dev/null @@ -1,4 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT src.* WHERE src.key < 100 -INSERT OVERWRITE TABLE dest2 SELECT src.key, src.value WHERE src.key >= 100 and src.key < 200 -INSERT OVERWRITE TABLE dest3 PARTITION(ds='2008-04-08', hr='12') SELECT src.key, 2 WHERE src.key >= 200 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input20.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input20.q deleted file mode 100644 index f30cf27017d96..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input20.q +++ /dev/null @@ -1,9 +0,0 @@ -FROM ( - FROM src - MAP src.key % 2, src.key % 5 - USING 'cat' - CLUSTER BY key -) tmap -REDUCE tmap.key, tmap.value -USING 'uniq -c | sed "s@^ *@@" | sed "s@\t@_@" | sed "s@ @\t@"' -AS key, value diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input3.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input3.q deleted file mode 100644 index fc53e94d39f02..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input3.q +++ /dev/null @@ -1,5 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest1 SELECT src.* WHERE src.key < 100 -INSERT OVERWRITE TABLE dest2 SELECT src.key, src.value WHERE src.key >= 100 and src.key < 200 -INSERT OVERWRITE TABLE dest3 PARTITION(ds='2008-04-08', hr='12') SELECT src.key, 2 WHERE src.key >= 200 and src.key < 300 -INSERT OVERWRITE DIRECTORY '../../../../build/contrib/hive/ql/test/data/warehouse/dest4.out' SELECT src.value WHERE src.key >= 300 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input4.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input4.q deleted file mode 100644 index 03e6de48faca3..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input4.q +++ /dev/null @@ -1,7 +0,0 @@ -FROM ( - FROM src - SELECT TRANSFORM(src.key, src.value) - USING '/bin/cat' AS (tkey, tvalue) - CLUSTER BY tkey -) tmap -INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue WHERE tmap.tkey < 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input5.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input5.q deleted file mode 100644 index a46abc75833f0..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input5.q +++ /dev/null @@ -1,7 +0,0 @@ -FROM ( - FROM src_thrift - SELECT TRANSFORM(src_thrift.lint, src_thrift.lintstring) - USING '/bin/cat' AS (tkey, tvalue) - CLUSTER BY tkey -) tmap -INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input6.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input6.q deleted file mode 100644 index d6f25a935ae7d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input6.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src1 -INSERT OVERWRITE TABLE dest1 SELECT src1.key, src1.value WHERE src1.key is null diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input7.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input7.q deleted file mode 100644 index 33a82953c26ed..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input7.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src1 -INSERT OVERWRITE TABLE dest1 SELECT NULL, src1.key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input8.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input8.q deleted file mode 100644 index 0843b9ba4e557..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input8.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src1 -SELECT 4 + NULL, src1.key - NULL, NULL + NULL diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input9.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input9.q deleted file mode 100644 index 2892f0b2dfc4f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input9.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src1 -INSERT OVERWRITE TABLE dest1 SELECT NULL, src1.key where NULL = NULL diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input_part1.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input_part1.q deleted file mode 100644 index d45d1cd0b47ec..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input_part1.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM srcpart -SELECT srcpart.key, srcpart.value, srcpart.hr, srcpart.ds WHERE srcpart.key < 100 and srcpart.ds = '2008-04-08' and srcpart.hr = '12' diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input_testsequencefile.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input_testsequencefile.q deleted file mode 100755 index cf9a092417e1d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input_testsequencefile.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src -INSERT OVERWRITE TABLE dest4_sequencefile SELECT src.key, src.value diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input_testxpath.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input_testxpath.q deleted file mode 100755 index 7699bff755520..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input_testxpath.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src_thrift -SELECT src_thrift.lint[1], src_thrift.lintstring[0].mystring, src_thrift.mstringstring['key_2'] diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input_testxpath2.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input_testxpath2.q deleted file mode 100644 index 08abaf4fad8da..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/input_testxpath2.q +++ /dev/null @@ -1,2 +0,0 @@ -FROM src_thrift -SELECT size(src_thrift.lint), size(src_thrift.lintstring), size(src_thrift.mstringstring) where src_thrift.lint IS NOT NULL AND NOT (src_thrift.mstringstring IS NULL) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/join1.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/join1.q deleted file mode 100644 index 739c39dd8f718..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/join1.q +++ /dev/null @@ -1,3 +0,0 @@ -FROM src src1 JOIN src src2 ON (src1.key = src2.key) -INSERT OVERWRITE TABLE dest1 SELECT src1.key, src2.value - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/join2.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/join2.q deleted file mode 100644 index a02d87f09f58c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/join2.q +++ /dev/null @@ -1,3 +0,0 @@ -FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key + src2.key = src3.key) -INSERT OVERWRITE TABLE dest1 SELECT src1.key, src3.value - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/join3.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/join3.q deleted file mode 100644 index b57c9569d7286..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/join3.q +++ /dev/null @@ -1,4 +0,0 @@ -FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key = src3.key) -INSERT OVERWRITE TABLE dest1 SELECT src1.key, src3.value - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/join4.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/join4.q deleted file mode 100644 index 2e5967fb7d854..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/join4.q +++ /dev/null @@ -1,14 +0,0 @@ -FROM ( - FROM - ( - FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 - ) a - LEFT OUTER JOIN - ( - FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 - ) b - ON (a.c1 = b.c3) - SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 -) c -SELECT c.c1, c.c2, c.c3, c.c4 - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/join5.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/join5.q deleted file mode 100644 index 63a38f554a24f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/join5.q +++ /dev/null @@ -1,15 +0,0 @@ -FROM ( - FROM - ( - FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 - ) a - RIGHT OUTER JOIN - ( - FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 - ) b - ON (a.c1 = b.c3) - SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 -) c -SELECT c.c1, c.c2, c.c3, c.c4 - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/join6.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/join6.q deleted file mode 100644 index 110451cf3039b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/join6.q +++ /dev/null @@ -1,16 +0,0 @@ -FROM ( - FROM - ( - FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 - ) a - FULL OUTER JOIN - ( - FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 - ) b - ON (a.c1 = b.c3) - SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 -) c -SELECT c.c1, c.c2, c.c3, c.c4 - - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/join7.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/join7.q deleted file mode 100644 index 65797b44a2cb1..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/join7.q +++ /dev/null @@ -1,21 +0,0 @@ -FROM ( - FROM - ( - FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 - ) a - FULL OUTER JOIN - ( - FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 - ) b - ON (a.c1 = b.c3) - LEFT OUTER JOIN - ( - FROM src src3 SELECT src3.key AS c5, src3.value AS c6 WHERE src3.key > 20 and src3.key < 25 - ) c - ON (a.c1 = c.c5) - SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4, c.c5 AS c5, c.c6 AS c6 -) c -SELECT c.c1, c.c2, c.c3, c.c4, c.c5, c.c6 - - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/join8.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/join8.q deleted file mode 100644 index d215b07a6720f..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/join8.q +++ /dev/null @@ -1,14 +0,0 @@ -FROM ( - FROM - ( - FROM src src1 SELECT src1.key AS c1, src1.value AS c2 WHERE src1.key > 10 and src1.key < 20 - ) a - LEFT OUTER JOIN - ( - FROM src src2 SELECT src2.key AS c3, src2.value AS c4 WHERE src2.key > 15 and src2.key < 25 - ) b - ON (a.c1 = b.c3) - SELECT a.c1 AS c1, a.c2 AS c2, b.c3 AS c3, b.c4 AS c4 -) c -SELECT c.c1, c.c2, c.c3, c.c4 where c.c3 IS NULL AND c.c1 IS NOT NULL - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample1.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/sample1.q deleted file mode 100644 index 3a168b999d707..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample1.q +++ /dev/null @@ -1,5 +0,0 @@ --- no input pruning, no sample filter -SELECT s.* -FROM srcpart TABLESAMPLE (BUCKET 1 OUT OF 1 ON rand()) s -WHERE s.ds='2008-04-08' and s.hr='11' - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample2.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/sample2.q deleted file mode 100644 index b505b896fa2c5..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample2.q +++ /dev/null @@ -1,4 +0,0 @@ --- input pruning, no sample filter --- default table sample columns -INSERT OVERWRITE TABLE dest1 SELECT s.* -FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 2) s diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample3.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/sample3.q deleted file mode 100644 index 42d5a2bbec34b..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample3.q +++ /dev/null @@ -1,4 +0,0 @@ --- sample columns not same as bucket columns --- no input pruning, sample filter -INSERT OVERWRITE TABLE dest1 SELECT s.* -- here's another test -FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 2 on key, value) s diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample4.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/sample4.q deleted file mode 100644 index 7b5ab03380ae1..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample4.q +++ /dev/null @@ -1,4 +0,0 @@ --- bucket column is the same as table sample --- No need for sample filter -INSERT OVERWRITE TABLE dest1 SELECT s.* -FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 2 on key) s diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample5.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/sample5.q deleted file mode 100644 index b9b48fdc71884..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample5.q +++ /dev/null @@ -1,3 +0,0 @@ --- no input pruning, sample filter -INSERT OVERWRITE TABLE dest1 SELECT s.* -- here's another test -FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 5 on key) s diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample6.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/sample6.q deleted file mode 100644 index 0ee026f0f368c..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample6.q +++ /dev/null @@ -1,3 +0,0 @@ --- both input pruning and sample filter -INSERT OVERWRITE TABLE dest1 SELECT s.* -FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 4 on key) s diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample7.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/sample7.q deleted file mode 100644 index f17ce105c3572..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/sample7.q +++ /dev/null @@ -1,4 +0,0 @@ --- both input pruning and sample filter -INSERT OVERWRITE TABLE dest1 SELECT s.* -FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 4 on key) s -WHERE s.key > 100 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/subq.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/subq.q deleted file mode 100644 index 6392dbcc4380d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/subq.q +++ /dev/null @@ -1,4 +0,0 @@ -FROM ( - FROM src select src.* WHERE src.key < 100 -) unioninput -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/union.out' SELECT unioninput.* diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/udf1.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/udf1.q deleted file mode 100644 index 2ecf46e742c30..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/udf1.q +++ /dev/null @@ -1,5 +0,0 @@ -FROM src SELECT 'a' LIKE '%a%', 'b' LIKE '%a%', 'ab' LIKE '%a%', 'ab' LIKE '%a_', - '%_' LIKE '\%\_', 'ab' LIKE '\%\_', 'ab' LIKE '_a%', 'ab' LIKE 'a', - '' RLIKE '.*', 'a' RLIKE '[ab]', '' RLIKE '[ab]', 'hadoop' RLIKE '[a-z]*', 'hadoop' RLIKE 'o*', - REGEXP_REPLACE('abc', 'b', 'c'), REGEXP_REPLACE('abc', 'z', 'a'), REGEXP_REPLACE('abbbb', 'bb', 'b'), REGEXP_REPLACE('hadoop', '(.)[a-z]*', '$1ive') - WHERE src.key = 86 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/udf4.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/udf4.q deleted file mode 100644 index f3a7598e17210..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/udf4.q +++ /dev/null @@ -1 +0,0 @@ -SELECT round(1.0), round(1.5), round(-1.5), floor(1.0), floor(1.5), floor(-1.5), sqrt(1.0), sqrt(-1.0), sqrt(0.0), ceil(1.0), ceil(1.5), ceil(-1.5), ceiling(1.0), rand(3), +3, -3, 1++2, 1+-2, ~1 FROM dest1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/udf6.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/udf6.q deleted file mode 100644 index 65791c41c1ff4..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/udf6.q +++ /dev/null @@ -1 +0,0 @@ -FROM src SELECT CONCAT('a', 'b'), IF(TRUE, 1 ,2) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/udf_case.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/udf_case.q deleted file mode 100644 index 0c86da2198699..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/udf_case.q +++ /dev/null @@ -1,10 +0,0 @@ -SELECT CASE 1 - WHEN 1 THEN 2 - WHEN 3 THEN 4 - ELSE 5 - END, - CASE 11 - WHEN 12 THEN 13 - WHEN 14 THEN 15 - END -FROM src LIMIT 1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/udf_when.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/udf_when.q deleted file mode 100644 index 99ed09990b874..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/udf_when.q +++ /dev/null @@ -1,10 +0,0 @@ -SELECT CASE - WHEN 1=1 THEN 2 - WHEN 3=5 THEN 4 - ELSE 5 - END, - CASE - WHEN 12=11 THEN 13 - WHEN 14=10 THEN 15 - END -FROM src LIMIT 1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/union.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/union.q deleted file mode 100644 index 6a6b9882aee71..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/positive/union.q +++ /dev/null @@ -1,6 +0,0 @@ -FROM ( - FROM src select src.key, src.value WHERE src.key < 100 - UNION ALL - FROM src SELECT src.* WHERE src.key > 100 -) unioninput -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/union.out' SELECT unioninput.* From 334c293ec0bcc2195d502c574ca40dbc4769d666 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 20 Apr 2016 16:32:38 -0700 Subject: [PATCH 67/69] [SPARK-14769][SQL] Create built-in functionality for variable substitution ## What changes were proposed in this pull request? In order to fully merge the Hive parser and the SQL parser, we'd need to support variable substitution in Spark. The implementation of the substitute algorithm is mostly copied from Hive, but I simplified the overall structure quite a bit and added more comprehensive test coverage. Note that this pull request does not yet use this functionality anywhere. ## How was this patch tested? Added VariableSubstitutionSuite for unit tests. Author: Reynold Xin Closes #12538 from rxin/SPARK-14769. --- .../apache/spark/sql/internal/SQLConf.scala | 16 +++ .../sql/internal/VariableSubstitution.scala | 121 ++++++++++++++++++ .../internal/VariableSubstitutionSuite.scala | 78 +++++++++++ 3 files changed, 215 insertions(+) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/internal/VariableSubstitution.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/internal/VariableSubstitutionSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 80e2c1986d758..27705520505a2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -457,6 +457,18 @@ object SQLConf { .booleanConf .createWithDefault(true) + val VARIABLE_SUBSTITUTE_ENABLED = + SQLConfigBuilder("spark.sql.variable.substitute") + .doc("This enables substitution using syntax like ${var} ${system:var} and ${env:var}.") + .booleanConf + .createWithDefault(true) + + val VARIABLE_SUBSTITUTE_DEPTH = + SQLConfigBuilder("spark.sql.variable.substitute.depth") + .doc("The maximum replacements the substitution engine will do.") + .intConf + .createWithDefault(40) + // TODO: This is still WIP and shouldn't be turned on without extensive test coverage val COLUMNAR_AGGREGATE_MAP_ENABLED = SQLConfigBuilder("spark.sql.codegen.aggregate.map.enabled") .internal() @@ -615,6 +627,10 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def columnarAggregateMapEnabled: Boolean = getConf(COLUMNAR_AGGREGATE_MAP_ENABLED) + def variableSubstituteEnabled: Boolean = getConf(VARIABLE_SUBSTITUTE_ENABLED) + + def variableSubstituteDepth: Int = getConf(VARIABLE_SUBSTITUTE_DEPTH) + override def orderByOrdinal: Boolean = getConf(ORDER_BY_ORDINAL) override def groupByOrdinal: Boolean = getConf(GROUP_BY_ORDINAL) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/VariableSubstitution.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/VariableSubstitution.scala new file mode 100644 index 0000000000000..0982f1d687161 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/VariableSubstitution.scala @@ -0,0 +1,121 @@ +/* + * 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.internal + +import java.util.regex.Pattern + +import org.apache.spark.sql.AnalysisException + +/** + * A helper class that enables substitution using syntax like + * `${var}`, `${system:var}` and `${env:var}`. + * + * Variable substitution is controlled by [[SQLConf.variableSubstituteEnabled]]. + */ +class VariableSubstitution(conf: SQLConf) { + + private val pattern = Pattern.compile("\\$\\{[^\\}\\$ ]+\\}") + + /** + * Given a query, does variable substitution and return the result. + */ + def substitute(input: String): String = { + // Note that this function is mostly copied from Hive's SystemVariables, so the style is + // very Java/Hive like. + if (input eq null) { + return null + } + + if (!conf.variableSubstituteEnabled) { + return input + } + + var eval = input + val depth = conf.variableSubstituteDepth + val builder = new StringBuilder + val m = pattern.matcher("") + + var s = 0 + while (s <= depth) { + m.reset(eval) + builder.setLength(0) + + var prev = 0 + var found = false + while (m.find(prev)) { + val group = m.group() + var substitute = substituteVariable(group.substring(2, group.length - 1)) + if (substitute.isEmpty) { + substitute = group + } else { + found = true + } + builder.append(eval.substring(prev, m.start())).append(substitute) + prev = m.end() + } + + if (!found) { + return eval + } + + builder.append(eval.substring(prev)) + eval = builder.toString + s += 1 + } + + if (s > depth) { + throw new AnalysisException( + "Variable substitution depth is deeper than " + depth + " for input " + input) + } else { + return eval + } + } + + /** + * Given a variable, replaces with the substitute value (default to ""). + */ + private def substituteVariable(variable: String): String = { + var value: String = null + + if (variable.startsWith("system:")) { + value = System.getProperty(variable.substring("system:".length())) + } + + if (value == null && variable.startsWith("env:")) { + value = System.getenv(variable.substring("env:".length())) + } + + if (value == null && conf != null && variable.startsWith("hiveconf:")) { + value = conf.getConfString(variable.substring("hiveconf:".length()), "") + } + + if (value == null && conf != null && variable.startsWith("sparkconf:")) { + value = conf.getConfString(variable.substring("sparkconf:".length()), "") + } + + if (value == null && conf != null && variable.startsWith("spark:")) { + value = conf.getConfString(variable.substring("spark:".length()), "") + } + + if (value == null && conf != null) { + value = conf.getConfString(variable, "") + } + + value + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/VariableSubstitutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/VariableSubstitutionSuite.scala new file mode 100644 index 0000000000000..deac95918bba5 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/VariableSubstitutionSuite.scala @@ -0,0 +1,78 @@ +/* + * 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.internal + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.AnalysisException + +class VariableSubstitutionSuite extends SparkFunSuite { + + private lazy val conf = new SQLConf + private lazy val sub = new VariableSubstitution(conf) + + test("system property") { + System.setProperty("varSubSuite.var", "abcd") + assert(sub.substitute("${system:varSubSuite.var}") == "abcd") + } + + test("environmental variables") { + assert(sub.substitute("${env:SPARK_TESTING}") == "1") + } + + test("Spark configuration variable") { + conf.setConfString("some-random-string-abcd", "1234abcd") + assert(sub.substitute("${hiveconf:some-random-string-abcd}") == "1234abcd") + assert(sub.substitute("${sparkconf:some-random-string-abcd}") == "1234abcd") + assert(sub.substitute("${spark:some-random-string-abcd}") == "1234abcd") + assert(sub.substitute("${some-random-string-abcd}") == "1234abcd") + } + + test("multiple substitutes") { + val q = "select ${bar} ${foo} ${doo} this is great" + conf.setConfString("bar", "1") + conf.setConfString("foo", "2") + conf.setConfString("doo", "3") + assert(sub.substitute(q) == "select 1 2 3 this is great") + } + + test("test nested substitutes") { + val q = "select ${bar} ${foo} this is great" + conf.setConfString("bar", "1") + conf.setConfString("foo", "${bar}") + assert(sub.substitute(q) == "select 1 1 this is great") + } + + test("depth limit") { + val q = "select ${bar} ${foo} ${doo}" + conf.setConfString(SQLConf.VARIABLE_SUBSTITUTE_DEPTH.key, "2") + + // This should be OK since it is not nested. + conf.setConfString("bar", "1") + conf.setConfString("foo", "2") + conf.setConfString("doo", "3") + assert(sub.substitute(q) == "select 1 2 3") + + // This should not be OK since it is nested in 3 levels. + conf.setConfString("bar", "1") + conf.setConfString("foo", "${bar}") + conf.setConfString("doo", "${foo}") + intercept[AnalysisException] { + sub.substitute(q) + } + } +} From f47dbf27fa034629fab12d0f3c89ab75edb03f86 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 20 Apr 2016 16:57:23 -0700 Subject: [PATCH 68/69] [SPARK-14602][YARN] Use SparkConf to propagate the list of cached files. This change avoids using the environment to pass this information, since with many jars it's easy to hit limits on certain OSes. Instead, it encodes the information into the Spark configuration propagated to the AM. The first problem that needed to be solved is a chicken & egg issue: the config file is distributed using the cache, and it needs to contain information about the files that are being distributed. To solve that, the code now treats the config archive especially, and uses slightly different code to distribute it, so that only its cache path needs to be saved to the config file. The second problem is that the extra information would show up in the Web UI, which made the environment tab even more noisy than it already is when lots of jars are listed. This is solved by two changes: the list of cached files is now read only once in the AM, and propagated down to the ExecutorRunnable code (which actually sends the list to the NMs when starting containers). The second change is to unset those config entries after the list is read, so that the SparkContext never sees them. Tested with both client and cluster mode by running "run-example SparkPi". This uploads a whole lot of files when run from a build dir (instead of a distribution, where the list is cleaned up), and I verified that the configs do not show up in the UI. Author: Marcelo Vanzin Closes #12487 from vanzin/SPARK-14602. --- .../scala/org/apache/spark/SparkConf.scala | 4 + .../spark/deploy/yarn/ApplicationMaster.scala | 62 +++++++++++++- .../org/apache/spark/deploy/yarn/Client.scala | 42 ++++++++-- .../yarn/ClientDistributedCacheManager.scala | 79 +++++++---------- .../spark/deploy/yarn/ExecutorRunnable.scala | 58 +------------ .../spark/deploy/yarn/YarnAllocator.scala | 6 +- .../spark/deploy/yarn/YarnRMClient.scala | 9 +- .../org/apache/spark/deploy/yarn/config.scala | 50 +++++++++++ .../ClientDistributedCacheManagerSuite.scala | 84 ++++++++----------- .../spark/deploy/yarn/ClientSuite.scala | 17 ++-- .../deploy/yarn/YarnAllocatorSuite.scala | 3 +- 11 files changed, 239 insertions(+), 175 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index acce6bc24fdf1..4384a06bd74d4 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -225,6 +225,10 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { this } + private[spark] def remove(entry: ConfigEntry[_]): SparkConf = { + remove(entry.key) + } + /** Get a parameter; throws a NoSuchElementException if it's not set */ def get(key: String): String = { getOption(key).getOrElse(throw new NoSuchElementException(key)) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index d447a59937be7..5bb63500c8f88 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -19,15 +19,17 @@ package org.apache.spark.deploy.yarn import java.io.{File, IOException} import java.lang.reflect.InvocationTargetException -import java.net.{Socket, URL} +import java.net.{Socket, URI, URL} import java.util.concurrent.atomic.AtomicReference +import scala.collection.mutable.HashMap import scala.util.control.NonFatal import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil @@ -120,6 +122,61 @@ private[spark] class ApplicationMaster( private var delegationTokenRenewerOption: Option[AMDelegationTokenRenewer] = None + // Load the list of localized files set by the client. This is used when launching executors, + // and is loaded here so that these configs don't pollute the Web UI's environment page in + // cluster mode. + private val localResources = { + logInfo("Preparing Local resources") + val resources = HashMap[String, LocalResource]() + + def setupDistributedCache( + file: String, + rtype: LocalResourceType, + timestamp: String, + size: String, + vis: String): Unit = { + val uri = new URI(file) + val amJarRsrc = Records.newRecord(classOf[LocalResource]) + amJarRsrc.setType(rtype) + amJarRsrc.setVisibility(LocalResourceVisibility.valueOf(vis)) + amJarRsrc.setResource(ConverterUtils.getYarnUrlFromURI(uri)) + amJarRsrc.setTimestamp(timestamp.toLong) + amJarRsrc.setSize(size.toLong) + + val fileName = Option(uri.getFragment()).getOrElse(new Path(uri).getName()) + resources(fileName) = amJarRsrc + } + + val distFiles = sparkConf.get(CACHED_FILES) + val fileSizes = sparkConf.get(CACHED_FILES_SIZES) + val timeStamps = sparkConf.get(CACHED_FILES_TIMESTAMPS) + val visibilities = sparkConf.get(CACHED_FILES_VISIBILITIES) + val resTypes = sparkConf.get(CACHED_FILES_TYPES) + + for (i <- 0 to distFiles.size - 1) { + val resType = LocalResourceType.valueOf(resTypes(i)) + setupDistributedCache(distFiles(i), resType, timeStamps(i).toString, fileSizes(i).toString, + visibilities(i)) + } + + // Distribute the conf archive to executors. + sparkConf.get(CACHED_CONF_ARCHIVE).foreach { uri => + val fs = FileSystem.get(new URI(uri), yarnConf) + val status = fs.getFileStatus(new Path(uri)) + setupDistributedCache(uri, LocalResourceType.ARCHIVE, status.getModificationTime().toString, + status.getLen.toString, LocalResourceVisibility.PRIVATE.name()) + } + + // Clean up the configuration so it doesn't show up in the Web UI (since it's really noisy). + CACHE_CONFIGS.foreach { e => + sparkConf.remove(e) + sys.props.remove(e.key) + } + + logInfo("Prepared Local resources " + resources) + resources.toMap + } + def getAttemptId(): ApplicationAttemptId = { client.getAttemptId() } @@ -292,7 +349,8 @@ private[spark] class ApplicationMaster( _sparkConf, uiAddress, historyAddress, - securityMgr) + securityMgr, + localResources) allocator.allocateResources() reporterThread = launchReporterThread() diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index ae5fb6bbd4f39..8b07dc3af4020 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -328,12 +328,14 @@ private[spark] class Client( private[yarn] def copyFileToRemote( destDir: Path, srcPath: Path, - replication: Short): Path = { + replication: Short, + force: Boolean = false, + destName: Option[String] = None): Path = { val destFs = destDir.getFileSystem(hadoopConf) val srcFs = srcPath.getFileSystem(hadoopConf) var destPath = srcPath - if (!compareFs(srcFs, destFs)) { - destPath = new Path(destDir, srcPath.getName()) + if (force || !compareFs(srcFs, destFs)) { + destPath = new Path(destDir, destName.getOrElse(srcPath.getName())) logInfo(s"Uploading resource $srcPath -> $destPath") FileUtil.copy(srcFs, srcPath, destFs, destPath, false, hadoopConf) destFs.setReplication(destPath, replication) @@ -553,12 +555,37 @@ private[spark] class Client( distribute(f, targetDir = targetDir) } - // Distribute an archive with Hadoop and Spark configuration for the AM and executors. + // Update the configuration with all the distributed files, minus the conf archive. The + // conf archive will be handled by the AM differently so that we avoid having to send + // this configuration by other means. See SPARK-14602 for one reason of why this is needed. + distCacheMgr.updateConfiguration(sparkConf) + + // Upload the conf archive to HDFS manually, and record its location in the configuration. + // This will allow the AM to know where the conf archive is in HDFS, so that it can be + // distributed to the containers. + // + // This code forces the archive to be copied, so that unit tests pass (since in that case both + // file systems are the same and the archive wouldn't normally be copied). In most (all?) + // deployments, the archive would be copied anyway, since it's a temp file in the local file + // system. + val remoteConfArchivePath = new Path(destDir, LOCALIZED_CONF_ARCHIVE) + val remoteFs = FileSystem.get(remoteConfArchivePath.toUri(), hadoopConf) + sparkConf.set(CACHED_CONF_ARCHIVE, remoteConfArchivePath.toString()) + + val localConfArchive = new Path(createConfArchive().toURI()) + copyFileToRemote(destDir, localConfArchive, replication, force = true, + destName = Some(LOCALIZED_CONF_ARCHIVE)) + val (_, confLocalizedPath) = distribute(createConfArchive().toURI().getPath(), resType = LocalResourceType.ARCHIVE, destName = Some(LOCALIZED_CONF_DIR)) require(confLocalizedPath != null) + // Clear the cache-related entries from the configuration to avoid them polluting the + // UI's environment page. This works for client mode; for cluster mode, this is handled + // by the AM. + CACHE_CONFIGS.foreach(sparkConf.remove) + localResources } @@ -787,10 +814,6 @@ private[spark] class Client( val launchEnv = setupLaunchEnv(appStagingDirPath, pySparkArchives) val localResources = prepareLocalResources(appStagingDirPath, pySparkArchives) - // Set the environment variables to be passed on to the executors. - distCacheMgr.setDistFilesEnv(launchEnv) - distCacheMgr.setDistArchivesEnv(launchEnv) - val amContainer = Records.newRecord(classOf[ContainerLaunchContext]) amContainer.setLocalResources(localResources.asJava) amContainer.setEnvironment(launchEnv.asJava) @@ -1150,6 +1173,9 @@ private object Client extends Logging { // Subdirectory where the user's Spark and Hadoop config files will be placed. val LOCALIZED_CONF_DIR = "__spark_conf__" + // File containing the conf archive in the AM. See prepareLocalResources(). + val LOCALIZED_CONF_ARCHIVE = LOCALIZED_CONF_DIR + ".zip" + // Name of the file in the conf archive containing Spark configuration. val SPARK_CONF_FILE = "__spark_conf__.properties" diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala index 869edf6c5b6af..dcc2288dd155a 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.yarn import java.net.URI -import scala.collection.mutable.{HashMap, LinkedHashMap, Map} +import scala.collection.mutable.{HashMap, ListBuffer, Map} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} @@ -27,17 +27,21 @@ import org.apache.hadoop.fs.permission.FsAction import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.util.{ConverterUtils, Records} +import org.apache.spark.SparkConf +import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.Logging +private case class CacheEntry( + uri: URI, + size: Long, + modTime: Long, + visibility: LocalResourceVisibility, + resType: LocalResourceType) + /** Client side methods to setup the Hadoop distributed cache */ private[spark] class ClientDistributedCacheManager() extends Logging { - // Mappings from remote URI to (file status, modification time, visibility) - private val distCacheFiles: Map[String, (String, String, String)] = - LinkedHashMap[String, (String, String, String)]() - private val distCacheArchives: Map[String, (String, String, String)] = - LinkedHashMap[String, (String, String, String)]() - + private val distCacheEntries = new ListBuffer[CacheEntry]() /** * Add a resource to the list of distributed cache resources. This list can @@ -72,61 +76,33 @@ private[spark] class ClientDistributedCacheManager() extends Logging { amJarRsrc.setResource(ConverterUtils.getYarnUrlFromPath(destPath)) amJarRsrc.setTimestamp(destStatus.getModificationTime()) amJarRsrc.setSize(destStatus.getLen()) - if (link == null || link.isEmpty()) throw new Exception("You must specify a valid link name") + require(link != null && link.nonEmpty, "You must specify a valid link name.") localResources(link) = amJarRsrc if (!appMasterOnly) { val uri = destPath.toUri() val pathURI = new URI(uri.getScheme(), uri.getAuthority(), uri.getPath(), null, link) - if (resourceType == LocalResourceType.FILE) { - distCacheFiles(pathURI.toString()) = (destStatus.getLen().toString(), - destStatus.getModificationTime().toString(), visibility.name()) - } else { - distCacheArchives(pathURI.toString()) = (destStatus.getLen().toString(), - destStatus.getModificationTime().toString(), visibility.name()) - } + distCacheEntries += CacheEntry(pathURI, destStatus.getLen(), destStatus.getModificationTime(), + visibility, resourceType) } } /** - * Adds the necessary cache file env variables to the env passed in + * Writes down information about cached files needed in executors to the given configuration. */ - def setDistFilesEnv(env: Map[String, String]): Unit = { - val (keys, tupleValues) = distCacheFiles.unzip - val (sizes, timeStamps, visibilities) = tupleValues.unzip3 - if (keys.size > 0) { - env("SPARK_YARN_CACHE_FILES") = keys.reduceLeft[String] { (acc, n) => acc + "," + n } - env("SPARK_YARN_CACHE_FILES_TIME_STAMPS") = - timeStamps.reduceLeft[String] { (acc, n) => acc + "," + n } - env("SPARK_YARN_CACHE_FILES_FILE_SIZES") = - sizes.reduceLeft[String] { (acc, n) => acc + "," + n } - env("SPARK_YARN_CACHE_FILES_VISIBILITIES") = - visibilities.reduceLeft[String] { (acc, n) => acc + "," + n } - } - } - - /** - * Adds the necessary cache archive env variables to the env passed in - */ - def setDistArchivesEnv(env: Map[String, String]): Unit = { - val (keys, tupleValues) = distCacheArchives.unzip - val (sizes, timeStamps, visibilities) = tupleValues.unzip3 - if (keys.size > 0) { - env("SPARK_YARN_CACHE_ARCHIVES") = keys.reduceLeft[String] { (acc, n) => acc + "," + n } - env("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") = - timeStamps.reduceLeft[String] { (acc, n) => acc + "," + n } - env("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") = - sizes.reduceLeft[String] { (acc, n) => acc + "," + n } - env("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") = - visibilities.reduceLeft[String] { (acc, n) => acc + "," + n } - } + def updateConfiguration(conf: SparkConf): Unit = { + conf.set(CACHED_FILES, distCacheEntries.map(_.uri.toString)) + conf.set(CACHED_FILES_SIZES, distCacheEntries.map(_.size)) + conf.set(CACHED_FILES_TIMESTAMPS, distCacheEntries.map(_.modTime)) + conf.set(CACHED_FILES_VISIBILITIES, distCacheEntries.map(_.visibility.name())) + conf.set(CACHED_FILES_TYPES, distCacheEntries.map(_.resType.name())) } /** * Returns the local resource visibility depending on the cache file permissions * @return LocalResourceVisibility */ - def getVisibility( + private[yarn] def getVisibility( conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): LocalResourceVisibility = { @@ -141,7 +117,7 @@ private[spark] class ClientDistributedCacheManager() extends Logging { * Returns a boolean to denote whether a cache file is visible to all (public) * @return true if the path in the uri is visible to all, false otherwise */ - def isPublic(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): Boolean = { + private def isPublic(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): Boolean = { val fs = FileSystem.get(uri, conf) val current = new Path(uri.getPath()) // the leaf level file should be readable by others @@ -157,7 +133,7 @@ private[spark] class ClientDistributedCacheManager() extends Logging { * the directory hierarchy to the given path) * @return true if all ancestors have the 'execute' permission set for all users */ - def ancestorsHaveExecutePermissions( + private def ancestorsHaveExecutePermissions( fs: FileSystem, path: Path, statCache: Map[URI, FileStatus]): Boolean = { @@ -177,7 +153,7 @@ private[spark] class ClientDistributedCacheManager() extends Logging { * imply the permission in the passed FsAction * @return true if the path in the uri is visible to all, false otherwise */ - def checkPermissionOfOther( + private def checkPermissionOfOther( fs: FileSystem, path: Path, action: FsAction, @@ -194,7 +170,10 @@ private[spark] class ClientDistributedCacheManager() extends Logging { * it in the cache, and returns the FileStatus. * @return FileStatus */ - def getFileStatus(fs: FileSystem, uri: URI, statCache: Map[URI, FileStatus]): FileStatus = { + private[yarn] def getFileStatus( + fs: FileSystem, + uri: URI, + statCache: Map[URI, FileStatus]): FileStatus = { val stat = statCache.get(uri) match { case Some(existstat) => existstat case None => diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index ef7908a3ef2ac..3d370e6d71426 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -18,7 +18,6 @@ package org.apache.spark.deploy.yarn import java.io.File -import java.net.URI import java.nio.ByteBuffer import java.util.Collections @@ -55,7 +54,8 @@ private[yarn] class ExecutorRunnable( executorMemory: Int, executorCores: Int, appId: String, - securityMgr: SecurityManager) + securityMgr: SecurityManager, + localResources: Map[String, LocalResource]) extends Runnable with Logging { var rpc: YarnRPC = YarnRPC.create(conf) @@ -77,9 +77,7 @@ private[yarn] class ExecutorRunnable( val ctx = Records.newRecord(classOf[ContainerLaunchContext]) .asInstanceOf[ContainerLaunchContext] - val localResources = prepareLocalResources ctx.setLocalResources(localResources.asJava) - ctx.setEnvironment(env.asJava) val credentials = UserGroupInformation.getCurrentUser().getCredentials() @@ -88,7 +86,7 @@ private[yarn] class ExecutorRunnable( ctx.setTokens(ByteBuffer.wrap(dob.getData())) val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores, - appId, localResources) + appId) logInfo(s""" |=============================================================================== @@ -136,8 +134,7 @@ private[yarn] class ExecutorRunnable( hostname: String, executorMemory: Int, executorCores: Int, - appId: String, - localResources: HashMap[String, LocalResource]): List[String] = { + appId: String): List[String] = { // Extra options for the JVM val javaOpts = ListBuffer[String]() @@ -239,53 +236,6 @@ private[yarn] class ExecutorRunnable( commands.map(s => if (s == null) "null" else s).toList } - private def setupDistributedCache( - file: String, - rtype: LocalResourceType, - localResources: HashMap[String, LocalResource], - timestamp: String, - size: String, - vis: String): Unit = { - val uri = new URI(file) - val amJarRsrc = Records.newRecord(classOf[LocalResource]) - amJarRsrc.setType(rtype) - amJarRsrc.setVisibility(LocalResourceVisibility.valueOf(vis)) - amJarRsrc.setResource(ConverterUtils.getYarnUrlFromURI(uri)) - amJarRsrc.setTimestamp(timestamp.toLong) - amJarRsrc.setSize(size.toLong) - localResources(uri.getFragment()) = amJarRsrc - } - - private def prepareLocalResources: HashMap[String, LocalResource] = { - logInfo("Preparing Local resources") - val localResources = HashMap[String, LocalResource]() - - if (System.getenv("SPARK_YARN_CACHE_FILES") != null) { - val timeStamps = System.getenv("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',') - val fileSizes = System.getenv("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',') - val distFiles = System.getenv("SPARK_YARN_CACHE_FILES").split(',') - val visibilities = System.getenv("SPARK_YARN_CACHE_FILES_VISIBILITIES").split(',') - for( i <- 0 to distFiles.length - 1) { - setupDistributedCache(distFiles(i), LocalResourceType.FILE, localResources, timeStamps(i), - fileSizes(i), visibilities(i)) - } - } - - if (System.getenv("SPARK_YARN_CACHE_ARCHIVES") != null) { - val timeStamps = System.getenv("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS").split(',') - val fileSizes = System.getenv("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES").split(',') - val distArchives = System.getenv("SPARK_YARN_CACHE_ARCHIVES").split(',') - val visibilities = System.getenv("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES").split(',') - for( i <- 0 to distArchives.length - 1) { - setupDistributedCache(distArchives(i), LocalResourceType.ARCHIVE, localResources, - timeStamps(i), fileSizes(i), visibilities(i)) - } - } - - logInfo("Prepared Local resources " + localResources) - localResources - } - private def prepareEnvironment(container: Container): HashMap[String, String] = { val env = new HashMap[String, String]() Client.populateClasspath(null, yarnConf, sparkConf, env, sparkConf.get(EXECUTOR_CLASS_PATH)) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 23742eab6268c..b59e6cff2f75c 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -63,7 +63,8 @@ private[yarn] class YarnAllocator( sparkConf: SparkConf, amClient: AMRMClient[ContainerRequest], appAttemptId: ApplicationAttemptId, - securityMgr: SecurityManager) + securityMgr: SecurityManager, + localResources: Map[String, LocalResource]) extends Logging { import YarnAllocator._ @@ -477,7 +478,8 @@ private[yarn] class YarnAllocator( executorMemory, executorCores, appAttemptId.getApplicationId.toString, - securityMgr) + securityMgr, + localResources) if (launchContainers) { logInfo("Launching ExecutorRunnable. driverUrl: %s, executorHostname: %s".format( driverUrl, executorHostname)) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala index e7f75446641cb..53df11eb66021 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala @@ -20,7 +20,6 @@ package org.apache.spark.deploy.yarn import java.util.{List => JList} import scala.collection.JavaConverters._ -import scala.collection.Map import scala.util.Try import org.apache.hadoop.conf.Configuration @@ -52,6 +51,8 @@ private[spark] class YarnRMClient extends Logging { * @param sparkConf The Spark configuration. * @param uiAddress Address of the SparkUI. * @param uiHistoryAddress Address of the application on the History Server. + * @param securityMgr The security manager. + * @param localResources Map with information about files distributed via YARN's cache. */ def register( driverUrl: String, @@ -60,7 +61,8 @@ private[spark] class YarnRMClient extends Logging { sparkConf: SparkConf, uiAddress: String, uiHistoryAddress: String, - securityMgr: SecurityManager + securityMgr: SecurityManager, + localResources: Map[String, LocalResource] ): YarnAllocator = { amClient = AMRMClient.createAMRMClient() amClient.init(conf) @@ -72,7 +74,8 @@ private[spark] class YarnRMClient extends Logging { amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) registered = true } - new YarnAllocator(driverUrl, driverRef, conf, sparkConf, amClient, getAttemptId(), securityMgr) + new YarnAllocator(driverUrl, driverRef, conf, sparkConf, amClient, getAttemptId(), securityMgr, + localResources) } /** diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala index edfbfc5d58d86..3816a84ab2e7c 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala @@ -261,4 +261,54 @@ package object config { .stringConf .toSequence .createOptional + + /* Configuration and cached file propagation. */ + + private[spark] val CACHED_FILES = ConfigBuilder("spark.yarn.cache.filenames") + .internal() + .stringConf + .toSequence + .createWithDefault(Nil) + + private[spark] val CACHED_FILES_SIZES = ConfigBuilder("spark.yarn.cache.sizes") + .internal() + .longConf + .toSequence + .createWithDefault(Nil) + + private[spark] val CACHED_FILES_TIMESTAMPS = ConfigBuilder("spark.yarn.cache.timestamps") + .internal() + .longConf + .toSequence + .createWithDefault(Nil) + + private[spark] val CACHED_FILES_VISIBILITIES = ConfigBuilder("spark.yarn.cache.visibilities") + .internal() + .stringConf + .toSequence + .createWithDefault(Nil) + + // Either "file" or "archive", for each file. + private[spark] val CACHED_FILES_TYPES = ConfigBuilder("spark.yarn.cache.types") + .internal() + .stringConf + .toSequence + .createWithDefault(Nil) + + // The location of the conf archive in HDFS. + private[spark] val CACHED_CONF_ARCHIVE = ConfigBuilder("spark.yarn.cache.confArchive") + .internal() + .stringConf + .createOptional + + // The list of cache-related config entries. This is used by Client and the AM to clean + // up the environment so that these settings do not appear on the web UI. + private[yarn] val CACHE_CONFIGS = Seq( + CACHED_FILES, + CACHED_FILES_SIZES, + CACHED_FILES_TIMESTAMPS, + CACHED_FILES_VISIBILITIES, + CACHED_FILES_TYPES, + CACHED_CONF_ARCHIVE) + } diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala index ac8f663df2fff..b696e080ce62f 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala @@ -33,7 +33,8 @@ import org.apache.hadoop.yarn.util.ConverterUtils import org.mockito.Mockito.when import org.scalatest.mock.MockitoSugar -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.yarn.config._ class ClientDistributedCacheManagerSuite extends SparkFunSuite with MockitoSugar { @@ -84,18 +85,13 @@ class ClientDistributedCacheManagerSuite extends SparkFunSuite with MockitoSugar assert(resource.getSize() === 0) assert(resource.getType() === LocalResourceType.FILE) - val env = new HashMap[String, String]() - distMgr.setDistFilesEnv(env) - assert(env("SPARK_YARN_CACHE_FILES") === "file:/foo.invalid.com:8080/tmp/testing#link") - assert(env("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === "0") - assert(env("SPARK_YARN_CACHE_FILES_FILE_SIZES") === "0") - assert(env("SPARK_YARN_CACHE_FILES_VISIBILITIES") === LocalResourceVisibility.PRIVATE.name()) - - distMgr.setDistArchivesEnv(env) - assert(env.get("SPARK_YARN_CACHE_ARCHIVES") === None) - assert(env.get("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === None) - assert(env.get("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === None) - assert(env.get("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === None) + val sparkConf = new SparkConf(false) + distMgr.updateConfiguration(sparkConf) + assert(sparkConf.get(CACHED_FILES) === Seq("file:/foo.invalid.com:8080/tmp/testing#link")) + assert(sparkConf.get(CACHED_FILES_TIMESTAMPS) === Seq(0L)) + assert(sparkConf.get(CACHED_FILES_SIZES) === Seq(0L)) + assert(sparkConf.get(CACHED_FILES_VISIBILITIES) === Seq(LocalResourceVisibility.PRIVATE.name())) + assert(sparkConf.get(CACHED_FILES_TYPES) === Seq(LocalResourceType.FILE.name())) // add another one and verify both there and order correct val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", @@ -111,20 +107,22 @@ class ClientDistributedCacheManagerSuite extends SparkFunSuite with MockitoSugar assert(resource2.getSize() === 20) assert(resource2.getType() === LocalResourceType.FILE) - val env2 = new HashMap[String, String]() - distMgr.setDistFilesEnv(env2) - val timestamps = env2("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',') - val files = env2("SPARK_YARN_CACHE_FILES").split(',') - val sizes = env2("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',') - val visibilities = env2("SPARK_YARN_CACHE_FILES_VISIBILITIES") .split(',') + val sparkConf2 = new SparkConf(false) + distMgr.updateConfiguration(sparkConf2) + + val files = sparkConf2.get(CACHED_FILES) + val sizes = sparkConf2.get(CACHED_FILES_SIZES) + val timestamps = sparkConf2.get(CACHED_FILES_TIMESTAMPS) + val visibilities = sparkConf2.get(CACHED_FILES_VISIBILITIES) + assert(files(0) === "file:/foo.invalid.com:8080/tmp/testing#link") - assert(timestamps(0) === "0") - assert(sizes(0) === "0") + assert(timestamps(0) === 0) + assert(sizes(0) === 0) assert(visibilities(0) === LocalResourceVisibility.PRIVATE.name()) assert(files(1) === "file:/foo.invalid.com:8080/tmp/testing2#link2") - assert(timestamps(1) === "10") - assert(sizes(1) === "20") + assert(timestamps(1) === 10) + assert(sizes(1) === 20) assert(visibilities(1) === LocalResourceVisibility.PRIVATE.name()) } @@ -165,18 +163,13 @@ class ClientDistributedCacheManagerSuite extends SparkFunSuite with MockitoSugar assert(resource.getSize() === 20) assert(resource.getType() === LocalResourceType.ARCHIVE) - val env = new HashMap[String, String]() - distMgr.setDistFilesEnv(env) - assert(env.get("SPARK_YARN_CACHE_FILES") === None) - assert(env.get("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === None) - assert(env.get("SPARK_YARN_CACHE_FILES_FILE_SIZES") === None) - assert(env.get("SPARK_YARN_CACHE_FILES_VISIBILITIES") === None) - - distMgr.setDistArchivesEnv(env) - assert(env.get("SPARK_YARN_CACHE_ARCHIVES") === None) - assert(env.get("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === None) - assert(env.get("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === None) - assert(env.get("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === None) + val sparkConf = new SparkConf(false) + distMgr.updateConfiguration(sparkConf) + assert(sparkConf.get(CACHED_FILES) === Nil) + assert(sparkConf.get(CACHED_FILES_TIMESTAMPS) === Nil) + assert(sparkConf.get(CACHED_FILES_SIZES) === Nil) + assert(sparkConf.get(CACHED_FILES_VISIBILITIES) === Nil) + assert(sparkConf.get(CACHED_FILES_TYPES) === Nil) } test("test addResource archive") { @@ -199,20 +192,13 @@ class ClientDistributedCacheManagerSuite extends SparkFunSuite with MockitoSugar assert(resource.getSize() === 20) assert(resource.getType() === LocalResourceType.ARCHIVE) - val env = new HashMap[String, String]() - - distMgr.setDistArchivesEnv(env) - assert(env("SPARK_YARN_CACHE_ARCHIVES") === "file:/foo.invalid.com:8080/tmp/testing#link") - assert(env("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === "10") - assert(env("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === "20") - assert(env("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === LocalResourceVisibility.PRIVATE.name()) - - distMgr.setDistFilesEnv(env) - assert(env.get("SPARK_YARN_CACHE_FILES") === None) - assert(env.get("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === None) - assert(env.get("SPARK_YARN_CACHE_FILES_FILE_SIZES") === None) - assert(env.get("SPARK_YARN_CACHE_FILES_VISIBILITIES") === None) + val sparkConf = new SparkConf(false) + distMgr.updateConfiguration(sparkConf) + assert(sparkConf.get(CACHED_FILES) === Seq("file:/foo.invalid.com:8080/tmp/testing#link")) + assert(sparkConf.get(CACHED_FILES_SIZES) === Seq(20L)) + assert(sparkConf.get(CACHED_FILES_TIMESTAMPS) === Seq(10L)) + assert(sparkConf.get(CACHED_FILES_VISIBILITIES) === Seq(LocalResourceVisibility.PRIVATE.name())) + assert(sparkConf.get(CACHED_FILES_TYPES) === Seq(LocalResourceType.ARCHIVE.name())) } - } diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index 06efd44b5df9c..f196a0d8ca32b 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -243,9 +243,12 @@ class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll assert(sparkConf.get(SPARK_JARS) === Some(Seq(s"local:${jar4.getPath()}", s"local:${single.getAbsolutePath()}/*"))) - verify(client).copyFileToRemote(any(classOf[Path]), meq(new Path(jar1.toURI())), anyShort()) - verify(client).copyFileToRemote(any(classOf[Path]), meq(new Path(jar2.toURI())), anyShort()) - verify(client).copyFileToRemote(any(classOf[Path]), meq(new Path(jar3.toURI())), anyShort()) + verify(client).copyFileToRemote(any(classOf[Path]), meq(new Path(jar1.toURI())), anyShort(), + anyBoolean(), any()) + verify(client).copyFileToRemote(any(classOf[Path]), meq(new Path(jar2.toURI())), anyShort(), + anyBoolean(), any()) + verify(client).copyFileToRemote(any(classOf[Path]), meq(new Path(jar3.toURI())), anyShort(), + anyBoolean(), any()) val cp = classpath(client) cp should contain (buildPath(PWD, LOCALIZED_LIB_DIR, "*")) @@ -262,7 +265,8 @@ class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll val client = createClient(sparkConf) client.prepareLocalResources(new Path(temp.getAbsolutePath()), Nil) - verify(client).copyFileToRemote(any(classOf[Path]), meq(new Path(archive.toURI())), anyShort()) + verify(client).copyFileToRemote(any(classOf[Path]), meq(new Path(archive.toURI())), anyShort(), + anyBoolean(), any()) classpath(client) should contain (buildPath(PWD, LOCALIZED_LIB_DIR, "*")) sparkConf.set(SPARK_ARCHIVE, LOCAL_SCHEME + ":" + archive.getPath()) @@ -281,7 +285,8 @@ class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll val sparkConf = new SparkConfWithEnv(Map("SPARK_HOME" -> temp.getAbsolutePath())) val client = createClient(sparkConf) client.prepareLocalResources(new Path(temp.getAbsolutePath()), Nil) - verify(client).copyFileToRemote(any(classOf[Path]), meq(new Path(jar.toURI())), anyShort()) + verify(client).copyFileToRemote(any(classOf[Path]), meq(new Path(jar.toURI())), anyShort(), + anyBoolean(), any()) classpath(client) should contain (buildPath(PWD, LOCALIZED_LIB_DIR, "*")) } @@ -382,7 +387,7 @@ class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll val clientArgs = new ClientArguments(args) val client = spy(new Client(clientArgs, conf, sparkConf)) doReturn(new Path("/")).when(client).copyFileToRemote(any(classOf[Path]), - any(classOf[Path]), anyShort()) + any(classOf[Path]), anyShort(), anyBoolean(), any()) client } diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index a641a6e73e853..784c6525e5577 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -104,7 +104,8 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter sparkConfClone, rmClient, appAttemptId, - new SecurityManager(sparkConf)) + new SecurityManager(sparkConf), + Map()) } def createContainer(host: String): Container = { From 24f338ba7b34df493dd49bbc354d08f5e3afbb85 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 20 Apr 2016 17:56:31 -0700 Subject: [PATCH 69/69] [SPARK-14775][SQL] Remove TestHiveSparkSession.rewritePaths ## What changes were proposed in this pull request? The path rewrite in TestHiveSparkSession is pretty hacky. I think we can remove those complexity and just do a string replacement when we read the query files in. This would remove the overloading of runNativeSql in TestHive, which will simplify the removal of Hive specific variable substitution. ## How was this patch tested? This is a small test refactoring to simplify test infrastructure. Author: Reynold Xin Closes #12543 from rxin/SPARK-14775. --- .../apache/spark/sql/hive/test/TestHive.scala | 18 ------------------ .../test/resources/hive-test-path-helper.txt | 1 + .../hive/execution/HiveComparisonTest.scala | 14 +++++++++++++- .../sql/hive/execution/HiveQueryFileTest.scala | 6 +++--- 4 files changed, 17 insertions(+), 22 deletions(-) create mode 100644 sql/hive/src/test/resources/hive-test-path-helper.txt diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index e629099086899..2bb13996c145c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -182,19 +182,6 @@ private[hive] class TestHiveSparkSession( Option(System.getenv(envVar)).map(new File(_)) } - /** - * Replaces relative paths to the parent directory "../" with hiveDevHome since this is how the - * hive test cases assume the system is set up. - */ - private[hive] def rewritePaths(cmd: String): String = - if (cmd.toUpperCase contains "LOAD DATA") { - val testDataLocation = - hiveDevHome.map(_.getCanonicalPath).getOrElse(inRepoTests.getCanonicalPath) - cmd.replaceAll("\\.\\./\\.\\./", testDataLocation + "/") - } else { - cmd - } - val hiveFilesTemp = File.createTempFile("catalystHiveFiles", "") hiveFilesTemp.delete() hiveFilesTemp.mkdir() @@ -566,11 +553,6 @@ private[hive] class TestHiveSessionState(sparkSession: TestHiveSparkSession) override def executePlan(plan: LogicalPlan): TestHiveQueryExecution = { new TestHiveQueryExecution(sparkSession, plan) } - - // Override so we can intercept relative paths and rewrite them to point at hive. - override def runNativeSql(sql: String): Seq[String] = { - super.runNativeSql(sparkSession.rewritePaths(substitutor.substitute(hiveconf, sql))) - } } diff --git a/sql/hive/src/test/resources/hive-test-path-helper.txt b/sql/hive/src/test/resources/hive-test-path-helper.txt new file mode 100644 index 0000000000000..356b131ea1149 --- /dev/null +++ b/sql/hive/src/test/resources/hive-test-path-helper.txt @@ -0,0 +1 @@ +This file is here so we can match on it and find the path to the current folder. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index bd46cb922e1ba..994dc4a2d28fc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -47,6 +47,17 @@ import org.apache.spark.sql.hive.test.{TestHive, TestHiveQueryExecution} abstract class HiveComparisonTest extends SparkFunSuite with BeforeAndAfterAll with GivenWhenThen { + /** + * Path to the test datasets. We find this by looking up "hive-test-path-helper.txt" file. + * + * Before we run the query in Spark, we replace "../../data" with this path. + */ + private val testDataPath: String = { + Thread.currentThread.getContextClassLoader + .getResource("hive-test-path-helper.txt") + .getPath.replace("/hive-test-path-helper.txt", "/data") + } + /** * When set, any cache files that result in test failures will be deleted. Used when the test * harness or hive have been updated thus requiring new golden answers to be computed for some @@ -386,7 +397,8 @@ abstract class HiveComparisonTest var query: TestHiveQueryExecution = null try { query = { - val originalQuery = new TestHiveQueryExecution(queryString) + val originalQuery = new TestHiveQueryExecution( + queryString.replace("../../data", testDataPath)) val containsCommands = originalQuery.analyzed.collectFirst { case _: Command => () case _: LogicalInsertIntoHiveTable => () diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala index f96c989c4614f..e772324a57ab8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala @@ -40,14 +40,14 @@ abstract class HiveQueryFileTest extends HiveComparisonTest { def testCases: Seq[(String, File)] - val runAll = + val runAll: Boolean = !(System.getProperty("spark.hive.alltests") == null) || runOnlyDirectories.nonEmpty || skipDirectories.nonEmpty - val whiteListProperty = "spark.hive.whitelist" + val whiteListProperty: String = "spark.hive.whitelist" // Allow the whiteList to be overridden by a system property - val realWhiteList = + val realWhiteList: Seq[String] = Option(System.getProperty(whiteListProperty)).map(_.split(",").toSeq).getOrElse(whiteList) // Go through all the test cases and add them to scala test.